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

[GitHub] [pulsar] poorbarcode opened a new pull request, #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Master Issue: #15370
   
   ### Motivation
   
   see #15370
   
   ### Modifications
   
   I will complete proposal #15370 with these pull requests( *current pull request is a part of step 7-1* ): 
   
   1. Write the batch transaction log handler: `TxnLogBufferedWriter`
   2. Configuration changes and protocol changes.
   3. Transaction log store enables the batch feature.
   4. Pending ack log store enables the batch feature.
   5. Supports dynamic configuration.
   6. Append admin API for transaction batch log and docs( admin and configuration doc ).
      GET /admin/v3/transactions/coordinatorStats
      GET /admin/v3/transactions/pendingAckStats/:tenant/:namespace:/:topic:/:subName
   7. Append metrics support for transaction batch log.
     7-1. Metrics of Txn Buffered Writer.
     7-2. `TransactionLog` and `PendingAckStore` enables the Metrics of Txn Buffered Writer
   
   ----
   
   ### The desired effect
   
   `TransactionLog` should create `TxnLogBufferedWriter` with params: 
   
   ```JSON
   {
     "metricsPrefix": "pulsar_txn_tc",
     "labelNames": "coordinatorId",
     "labelValues": "1"
   }
   ```
   
   The  metrics output of `TransactionLog` will like this: 
   
   ```
   # A metrics for how many batches were triggered due to threshold "batchedWriteMaxRecords".
   # TYPE pulsar_txn_tc_batched_log_batched_log_triggering_count_by_records Counter
   pulsar_txn_tc_batched_log_batched_log_triggering_count_by_records{coordinatorId="1"} 15
   ...
   ...
   ...
   # pulsar_txn_tc_batched_log_records_count_per_entry A metrics for how many records in per batch written by the component[pulsar_txn_tc] per batch.
   # TYPE pulsar_txn_tc_batched_log_records_count_per_entry Histogram
   pulsar_txn_tc_batched_log_records_count_per_entry_bucket{coordinatorId="1", le="10"} 1
   pulsar_txn_tc_batched_log_records_count_per_entry_bucket{coordinatorId="1", le="50"} 3
   pulsar_txn_tc_batched_log_records_count_per_entry_bucket{coordinatorId="1", le="100"} 5
   pulsar_txn_tc_batched_log_records_count_per_entry_bucket{coordinatorId="1", le="500"} 10
   pulsar_txn_tc_batched_log_records_count_per_entry_bucket{coordinatorId="1", le="1000"} 10
   pulsar_txn_tc_batched_log_records_count_per_entry_bucket{coordinatorId="1", le="+Inf"} 10
   pulsar_txn_tc_batched_log_records_count_per_entry_count{coordinatorId="1", le="+Inf"} 10
   pulsar_txn_tc_batched_log_records_count_per_entry_sum{coordinatorId="1", le="+Inf"} 5432
   ```
   
   `PendingAckStore` is the same. But all the PendingackStores will not differentiate the Subscription labels (because there are too many)
   
   ----
   
   ### Manage the registered collectors ourselves.
   
   To build Metrics Stat, we need to execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Logs share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like this:
   
   ```java
   public Collector getRegistedCollector(String name);
   ```
   
   and it will throw IllegalArgumentException when we registering collector with the same name more than once, see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.
   
   ----
   
   #### Holds the `Collector.child` by each Metrics stat instance
   
   To save the overhead of `collector.labels(labelValues)`, we make each Metrics Stat hold a reference of `Collector.child`, because this method is not light enough:
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/SimpleCollector.java#L63-L80
   
   ----
   
   #### Code will be removed in the next PR (7-2)
   
   In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables the batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR. This is compatible only with split PR, making each PR have less code
   
   ### Documentation
   
   
   - [ ] `doc-required` 
     
   - [x] `doc-not-needed` 
     
   - [ ] `doc` 
   
   - [ ] `doc-complete`


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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);

Review Comment:
   I don't understand. What do you mean by refresh?



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -546,10 +552,9 @@ public void addCallback(AddDataCallback callback, Object ctx){
         }
     }
 
-    /** Callback for batch write BK. **/
-    private final BufferedAddEntryCallback bufferedAddEntryCallback = new BufferedAddEntryCallback();
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();
 
-    private class BufferedAddEntryCallback implements AsyncCallbacks.AddEntryCallback{
+    private class BookKeeperBatchedWriteCallback implements AsyncCallbacks.AddEntryCallback{

Review Comment:
   Yes, `BookKeeperBatchedWriteCallback` accessed the method `failureCallbackByContextAndRecycle` and `managedLedger`.  I tried make `failureCallbackByContextAndRecycle` to `static`, but this method also used `managedLedger`. both of them used `managedLedger` to print logs.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);
                     return;
                 }
                 // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
                 // fail.
-                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                failureCallbackByContextAndRecycle(flushContext,

Review Comment:
   Now `MlTransactionLogImpl` and `MlPendingAckStore` only have special treatment for `ManagedLedgerException`, If we create a new Exception, we also should change `MlTransactionLogImpl` and `MlPendingAckStore` for adaptation.Since this PR is already too large, I will create a new Exception in the following PR to solve it(same as https://github.com/apache/pulsar/pull/16758#discussion_r945521841). Thanks
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it only used in the constructor.
   
   Yes, we need.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we need to get the `Collector` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like `collectorRegistry.getRegistedCollector(String name)`, and it will throws IllegalArgumentException when we registering collector with the same name more than once, see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it is only used in the constructor.
   
   Yes, we need it.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`.
   
   (<strong> High light</strong>)
    However, the CollectorRegistry does not provide an API like below, and it will throw IllegalArgumentException when we registering collector with the same name more than once.
   
   ```java
   public Collector getRegistedCollector(String name);
   ```
   
   Throws IllegalArgumentException at this line:
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   
   > Here is what I suggest IMO to make it a bit simpler: 1-6
   
   I've taken care of all the suggestions, could you review this PR again?
   


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

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

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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);

Review Comment:
   Same as the above comment



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   @poorbarcode But it will break the order, no? If you have 3 records need to write to the managed ledger, A, B, and C. If A fails here, B and C will continue to write. As you said the request hold in the memory queue, the next round A will be retry.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,6 +153,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
+        if (batchedWriteMaxRecords <= 1 && batchEnabled){
+            log.warn("The current maximum number of records per batch is set to 1. Disabling Batch will improve"

Review Comment:
   Hi @asafm @codelipenghui 
   
   Can we add validation in the following PR, the validation will trigger test rewrite, and there is already a plan to rewrite `TxnLogBufferedWriterTest.testMainProcess`, I will add validation in that PR



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -89,15 +89,15 @@
     /**
      * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
      */
-    private final int batchedWriteMaxRecords;
+    final int batchedWriteMaxRecords;

Review Comment:
   What's the motivation for the removal of private from these three variables?
   
   These changes have been restored. Thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;

Review Comment:
   Good idea. Already change the name of the variable to `recordsPerBatchHistogram`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +620,322 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends JsonDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * 1. Verify Transaction buffered writer stats correct when enabled batch feature. Exclusive "triggerByForceFlush",
+     *    this property verified by {@link #testMetricsStatsWhenForceFlush()}.
+     * 2. Verify metrics will be release after {@link TxnLogBufferedWriterMetricsStats#clone()}.
+     */
+    @Test
+    public void testMetricsStatsWhenEnabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger to get the count of refresh event.
+        AtomicInteger refreshCount = new AtomicInteger();
+        String managedLedgerName = "-";
+        ManagedLedger managedLedger = Mockito.mock(ManagedLedger.class);
+        Mockito.when(managedLedger.getName()).thenReturn(managedLedgerName);
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                refreshCount.incrementAndGet();
+                AsyncCallbacks.AddEntryCallback callback =
+                        (AsyncCallbacks.AddEntryCallback) invocation.getArguments()[1];
+                callback.addComplete(PositionImpl.get(1,1), (ByteBuf)invocation.getArguments()[0],
+                        invocation.getArguments()[2]);
+                return null;
+            }
+        }).when(managedLedger).asyncAddEntry(Mockito.any(ByteBuf.class), Mockito.any(), Mockito.any());
+        // Mock addDataCallbackCount to get the count of add data finish count;
+        AtomicInteger addDataCallbackFinishCount = new AtomicInteger();
+        AtomicInteger addDataCallbackFailureCount = new AtomicInteger();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object context) {
+                addDataCallbackFinishCount.incrementAndGet();
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+                addDataCallbackFailureCount.incrementAndGet();
+            }
+        };
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, true, metricsStats);
+        // Add some data.
+        int writeCount = 3000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        Assert.assertEquals(getCounterValue(String.format("%s_batched_log_triggering_count_by_force", metricsPrefix)), 0D);
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_records", metricsPrefix))
+                + getCounterValue(String.format("%s_batched_log_triggering_count_by_size", metricsPrefix))
+                + getCounterValue(String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix)),
+                (double)refreshCount.get());
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                refreshCount.get());
+        Assert.assertEquals(
+                getHistogramSum(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                writeCount);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                refreshCount.get());
+        Assert.assertEquals(
+                getHistogramSum(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                dataSerializer.getTotalSize());
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix)),
+                refreshCount.get());
+        /**
+         * Assert all metrics will be released after {@link TxnLogBufferedWriter#close()}
+         *   1. Register another {@link TxnLogBufferedWriter}
+         *   2. Close first {@link TxnLogBufferedWriter}, verify the labels of metrics will be released after
+         *      {@link TxnLogBufferedWriter#close()}
+         *   3. Close second {@link TxnLogBufferedWriter},verify all metrics will be released after all
+         *      {@link TxnLogBufferedWriter#close()}
+         */
+        TxnLogBufferedWriterMetricsStats anotherMetricsStat = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, new String[]{"2"}, CollectorRegistry.defaultRegistry);
+        TxnLogBufferedWriter<Integer> anotherTxnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, true, anotherMetricsStat);
+        anotherTxnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount + 1
+        );
+        // Close first-writer, verify the labels will be released after writer-close.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> getHistogramCount(
+                        String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix)) == 0
+        );
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_records", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_size", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                0D
+        );
+        // Close second-writer, verify all metrics will be released after all writer-close.
+        anotherTxnLogBufferedWriter.close();
+        anotherMetricsStat.close();
+        // cleanup.
+        transactionTimer.stop();
+        managedLedger.close();
+        orderedExecutor.shutdown();
+    }
+
+    private double getCounterValue(String name) {
+        Double d = CollectorRegistry.defaultRegistry.getSampleValue(
+                name + "_total",
+                metricsLabelNames,
+                metricsLabelValues);
+        return d == null ? 0: d.doubleValue();
+    }
+
+    private double getHistogramCount(String name) {
+        Double d = CollectorRegistry.defaultRegistry.getSampleValue(
+                name + "_count",
+                metricsLabelNames,
+                metricsLabelValues);
+        return d == null ? 0: d.doubleValue();
+    }
+
+    private double getHistogramSum(String name) {
+        Double d = CollectorRegistry.defaultRegistry.getSampleValue(
+                name + "_sum",
+                metricsLabelNames,
+                metricsLabelValues);
+        return d == null ? 0: d.doubleValue();
+    }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Mock addDataCallbackCount to get the count of add data finish count;
+        AtomicInteger addDataCallbackFinishCount = new AtomicInteger();
+        AtomicInteger addDataCallbackFailureCount = new AtomicInteger();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object context) {
+                addDataCallbackFinishCount.incrementAndGet();
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+                addDataCallbackFailureCount.incrementAndGet();
+            }
+        };
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        Assert.assertEquals(getCounterValue(String.format("%s_batched_log_triggering_count_by_force", metricsPrefix)), 0D);
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_records", metricsPrefix))
+                    + getCounterValue(String.format("%s_batched_log_triggering_count_by_size", metricsPrefix))
+                    + getCounterValue(String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix)),
+                0D);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                0D);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                0D);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix)),
+                0D);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * Test {@link TxnLogBufferedWriterMetricsStats#triggerFlushByForce(int, long, long)}.
+     */
+    @Test
+    public void testMetricsStatsWhenForceFlush() throws Exception {

Review Comment:
   > If you remove public force flush then this test can be removed?
   
   No, I will rename this method to `testMetricsStatsWhenTriggeredLargeSingleData`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +620,322 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends JsonDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * 1. Verify Transaction buffered writer stats correct when enabled batch feature. Exclusive "triggerByForceFlush",
+     *    this property verified by {@link #testMetricsStatsWhenForceFlush()}.
+     * 2. Verify metrics will be release after {@link TxnLogBufferedWriterMetricsStats#clone()}.
+     */
+    @Test
+    public void testMetricsStatsWhenEnabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger to get the count of refresh event.
+        AtomicInteger refreshCount = new AtomicInteger();

Review Comment:
   Good idea, I have already changed the name of this variable to `batchFlushCount `



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm @codelipenghui 
   
   I've taken care of all the comments, could you review this PR again?


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);

Review Comment:
   > Same as the above comment
   
   `doFlush` failure will not break the order,  there is no problem 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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   What you wrote is there two 2 separate threads using asyncAddArgsList no?



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;

Review Comment:
   Yes. already fix



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;

Review Comment:
    already 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] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;

Review Comment:
   Good idea. Already change the name of the variable to `batchSizeBytesMetric`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;

Review Comment:
   Good idea. Already change the name of the variable to`batchSizeBytesHistogram`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;

Review Comment:
   Good idea, already change the name of the variable to `batchFlushTriggeredByMaxRecordsMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {
+                    metricsStats.triggerFlushByForce(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
                 doFlush();
                 return;
             }
             if (byScheduleThreads) {
+                if (metricsStats != null) {
+                    metricsStats.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
                 doFlush();
                 return;
             }
             AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
             if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
+                if (metricsStats != null) {
+                    metricsStats.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);

Review Comment:
   Yes,  absolutely right



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some comments here.
   I didn't want to support disabled metrics stat in the design, even if the Txn Buffered Writer disabled the batch feature. This constructor without `metricsStats` and "null checks" will be removed in the next PR. This is only compatible to split the PR, making each PR has less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some code-comments here.
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR. This is compatible only with split PR, making each PR have less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some code comments in this PR.
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR. This is compatible only with split PR, making each PR have less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.

Review Comment:
   Good suggestion, I will use this technique. And already fixed in this PR.



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.

Review Comment:
   Instead of writing this comment, just write the code to look like the comment:
   ```
   int dataLength = dataSerializer.getSerializedSize(data);
           if (dataLength >= batchedWriteMaxSize){
   ```
   
   Whenever you stop to read a comment, you break your concentration. 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            if (flushContext.asyncAddArgsList.isEmpty()) {
+                return;
+            }
+            if (metrics != null) {
+                metrics.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            // Start the next timing task.
+            nextTimingTrigger();
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
+            if (metrics != null) {
+                metrics.triggerFlushByRecordsCount(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize) {
+            if (metrics != null) {
+                metrics.triggerFlushByBytesSize(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+        }
+    }
+
+    /**

Review Comment:
   You already wrote that comment above. Only add information that hasn't been added there



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            if (flushContext.asyncAddArgsList.isEmpty()) {
+                return;
+            }
+            if (metrics != null) {
+                metrics.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            // Start the next timing task.
+            nextTimingTrigger();
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   Why do write `this.` ?



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".

Review Comment:
   Comment not needed - you have method name for it



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        verifyTheCounterMetrics(writeCount / 2,0,0,0);
+        verifyTheHistogramMetrics(batchFlushCounter.get(), writeCount, writeCount * 4);

Review Comment:
   Now I need to create below what each parameter is named
   what is 4? why *4? Tie together to the params of the bufered writer
   
   ```
   verifyTriggeredByMaxSizeMetricEqualsTo(mockAddDataCallback.flushCounter.get())
   ```
   Do this call for each you verify



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);

Review Comment:
   import static assert
   Wonder why we don't use AssertJ as in `assertThat(addDataCallbackFailureCount.get()).equalsTo(0)`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/
+    private final TxnLogBufferedWriterMetricsStats metricsStats;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor, Timer timer,

Review Comment:
   Ok. So the `if metrics != null` will go away as well, right?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.

Review Comment:
   Change the code below to look like the comment
   ```
   queue.add(data);
   ```
   
   This suggestion applies to all comments in this class.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.

Review Comment:
   You method name looks exactly like your comment, just with spaces - it's redundant.
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            if (flushContext.asyncAddArgsList.isEmpty()) {
+                return;
+            }
+            if (metrics != null) {
+                metrics.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            // Start the next timing task.
+            nextTimingTrigger();
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
+            if (metrics != null) {
+                metrics.triggerFlushByRecordsCount(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize) {
+            if (metrics != null) {
+                metrics.triggerFlushByBytesSize(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+        }
+    }
+
+    /**
+     * If method {@link #asyncAddData(Object, AddDataCallback, Object)} accept a request that {@param data} is too
+     * large (larger than {@link #batchedWriteMaxSize}), then two flushes:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Directly write the large data to BK, this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private void trigFlushByLargeSingleData(T data, AddDataCallback callback, Object ctx){
+        if (!flushContext.asyncAddArgsList.isEmpty()) {
+            if (metrics != null) {
+                metrics.triggerFlushByLargeSingleData(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+        }
+        ByteBuf byteBuf = dataSerializer.serialize(data);

Review Comment:
   I would put those two lines back in `internalAsyncAddData` where you called `trigFlushByLargeSingleData`, since this method does one thing: trigger a flush as the name suggests.



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.

Review Comment:
   Convert this comment into code:
   ```java
   MockManagedLedger mockManagedLedger = createMockManagedLedgerWithCounters()
   ```
   
   where it's a simple data class
   ```java
   class MockManagedLedger {
       ManagedLedger managedLedger;
       AtomicInteger     batchFlushCounter
   }
   ```      
   
   Then you can delete your comment and delete
   ```java
           ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
           AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   I don't understand this entire explanation. Why does a metric indicating you flushed due to accepting a single large record obscure anything?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/

Review Comment:
   I still see the comment. 



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =

Review Comment:
   Same here
   ```
   class MockAddDataCallback {
       addDataCallback
       finishCount
       failureCount
   }
   ```



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/

Review Comment:
   Comment not needed



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        verifyTheCounterMetrics(writeCount / 2,0,0,0);

Review Comment:
   Make it explicit: What is 2? Why 2?
   ```
   writeCount / maxRecordPerBatch
   ```



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.

Review Comment:
   Push this into a method since it's not really interesting - the parameters are the interesting ones - the special ones.
   
   ```
   maxBatchSize=2
   ...
   createBufferedWriter(maxBatchSize, ...)
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   >But the Transaction Log Provider will hold all the Collector of Txn Buffered Writer; this is confusing
   
   Can you explain this further? I didn't understand how the provider would hold all the Collectors (metrics). It should only be to a single instance of the metric class.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();

Review Comment:
   > Where are you checking actualBatchFlushCount == 1 ?
   
   Since there has `sleep` at the front, so will flush many times, maybe 40-60 times. and we can only use `mockedManagedLedger.writeCounter.get()` to get the actual count of flush events



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > As I understand, this only happens once per instance of MLTransactionLogImpl. So I don't understand why you want to pass metric by metric. Something doesn't add up here.
   
   > I guess my main question is: how many MLTransactionLogImpl are there in a single broker process? More than 1?
   
   Yes, maybe more than 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] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   
   > IMO this explanation is confusing and doesn't serve what you truly want to convey. I would add the following explanation:
   
   Good suggestion. already fixed, thanks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it is only used in the constructor.
   
   Yes, we need it.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like this(<strong> High light</strong>):
   
   ```java
   public Collector getRegistedCollector(String name);
   ```
   
   and it will throw IllegalArgumentException when we registering collector with the same name more than once(<strong> High light</strong>), see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;

Review Comment:
   Good idea. The name of the variable has been changed to`batchSizeBytesMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByRecords;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    Counter pulsarBatchedLogTriggeringCountBySize;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByDelayTime;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    Counter pulsarBatchedLogTriggeringCountByForce;
+
+    public static synchronized TxnLogBufferedWriterMetricsStats getInstance(
+                                                            TxnLogBufferedWriterMetricsDefinition metricsDefinition){
+        // Mark who references stats.
+        METRICS_INSTANCE_REFERENCE.computeIfAbsent(metricsDefinition.getComponent(), n -> new ArrayList<>());
+        METRICS_INSTANCE_REFERENCE.get(metricsDefinition.getComponent()).add(metricsDefinition);
+        // Get or create stats.
+        return METRICS_REGISTRY.computeIfAbsent(

Review Comment:
   U are right. 
   
   All the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} should use the same {@param labelNames}, if not, the first registered {@param metricsPrefix} will be used, and if the number of {@param labelNames} differs from the first, an IllegalArgumentException is thrown.
   
   Now neither TC nor Txn Pending Ack Store is going to break this rule: "All the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} should use the same {@param labelNames}"
   
   I have append this comment to the constructor of `TxnLogBufferedWriterMetricsStats`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea. Already change the name of variable to `recordsPerBatchMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   Hi @asafm These comments are mainly to describe this:
   
   For E.g. we write these data:
   
   ```java
   // max batch size = 100, max batch records = 5, max delay = 10 milliseconds
   write data_01{ size = 2}
   write data_02{ size = 2}
   write data_03{ size = 2}
   write data_04{ size = 2}
   write data_05{ size = 2}
   write data_06{ size = 111}
   ```
   We have two plans of metrics: include `data_06` or not.
   
   - Exclude `data_06`
     - avg ( batch records count ) = 5
     - avg ( batch size) = 10
   - Include `data_06`
     - avg ( batch records count ) = 3
     - avg ( batch size) = 60
   
   The first plan(exclude data_06) presents metrics that make more sense for adjusting the threshold.
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   For E.g. we write these data:
   
   ```java
   // max batch size = 100, max batch records = 5, max delay = 10 milliseconds
   write data_01{ size = 2}
   write data_02{ size = 2}
   write data_03{ size = 2}
   write data_04{ size = 2}
   write data_05{ size = 2}
   write data_06{ size = 111}
   ```
   We have two plans of metrics: include `data_06` or not.
   
   - Exclude `data_06`
     - avg ( batch records count ) = 5
     - avg ( batch size) = 10
   - Include `data_06`
     - avg ( batch records count ) = 3
     - avg ( batch size) = 60
   
   The first plan(exclude data_06) presents metrics that make more sense for adjusting the threshold.
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Ok, if more than one, then the design must change. I thought the whole idea was that you have a single instance of metrics per metric prefix.
   > If not, after much thought I suggest the following:
   
   ```
   abstract class BufferMetrics {
   	protected abstract void observeRecordsPerBatch(int)
   	protected abstract void incFlushTriggeredByMaxRecords(int)
   }
   
   MLTransactionMetadataStoreBufferedWriterMetrics extends BufferMetrics {
   	static private Histogram recordsPerBatchMetric = new Histogram.Builder()
                           .name("pulsar_tx_store_bufferedwriter_batch_record_count")
                           .labelNames(new String[]{"txCoordinatorId"})
                           .help("Records per batch histogram")
                           .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
                           .register(registry));
   	
       private Histogram.Child recordsPerBatchHistogram;                      
   
   
   	public MLTransactionMetadataStoreBufferedWriterMetrics(String txCoordinatorId) {
   	    recordsPerBatchHistogram = recordsPerBatchHistogram.labels(txCoordinatorId)
   
   	}
        
          protected observeRecordsPerBatch(value) {
               recordsPerBatchHistogram.observe(value)       
          }
   
   }
   ```
   
   > Another approach which I disliked a bit, but it's still ok:
   > Add to Pulsar Common:
   
   ```
   class PrometheusRegistryChecker {
        static defaultMetricRegistryNameToCollector = new HashMap<String, Collector>()
   
        static Collector registerIfNotExists(collector) {}
   }
   ```
   
   > Like FunctionCollectorRegistryImpl
   
   I prefer the second implementation because if we need to define many variables of Collector type. And `Map` is really just another representation of multiple variables.
   
   Now `common` has not the dependency: `prometheus-client`



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -546,10 +552,9 @@ public void addCallback(AddDataCallback callback, Object ctx){
         }
     }
 
-    /** Callback for batch write BK. **/
-    private final BufferedAddEntryCallback bufferedAddEntryCallback = new BufferedAddEntryCallback();
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();
 
-    private class BufferedAddEntryCallback implements AsyncCallbacks.AddEntryCallback{
+    private class BookKeeperBatchedWriteCallback implements AsyncCallbacks.AddEntryCallback{

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] asafm commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   @tjiuming @poorbarcode I actually found this pattern already in Pulsar. See `ComponentStatsManager`


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   rebase master


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm  @tjiuming 
   
   I have removed the static variable `Collector_cache` from Class `TxnLogBufferedWriterMetricsStat`, because I changed the design: 
   
   - Transaction Log Provider will create many instances of ML Transaction Log, each with its own new instance of Txn buffered writer.
   - All ML Transaction Logs will use the same shared `TxnLogBufferedWriterMetricsStat` instance.
   
   Same with Pending Ack Store. 
   
   After that, `TxnLogBufferedWriterMetricsStats` of the same metrics prefix will only create once, so the static variable `Collector_cache` is unnecessary.


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,6 +153,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
+        if (batchedWriteMaxRecords <= 1 && batchEnabled){
+            log.warn("The current maximum number of records per batch is set to 1. Disabling Batch will improve"

Review Comment:
   Hi @asafm @codelipenghui 
   
   Already add the validation in the pulsar service constructor.



-- 
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] tjiuming commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   Does this Class singleton?
   If singleton, it's better to:
   ```
   private static TxnLogBufferedWriterMetricsStats stats;
   public static TxnLogBufferedWriterMetricsStats create(Args... args) {
       if (stats == null) {
            stats = new TxnLogBufferedWriterMetricsStats(args);
       }
       return stats;
   }
   ``` 
   if not a singleton, 
   ```
          recordsPerBatchMetric = new Histogram.Builder()
                           .name(recordsPerBatchMetricName)
                           .labelNames(labelNames)
                           .help("Records per batch histogram")
                           .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
                           .register(registry);
   ```
   may lead to a Exception if `registry` is always a same instance.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   Hi @tjiuming @asafm 
   
   Yes, this has been discussed.
   
   TxnLogBufferedWriterMetricsStats only two instances, they are holds by `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider`.  In the following PR, there will be changes like this: 
   
   - Append modifier `abstract` to `TxnLogBufferedWriterMetricsStats`
   - Create an internal class in `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider` like this:
   
   ```java
   private static class MLTransactionMetadataStoreBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTransactionMetadataStoreBufferedWriterMetrics() {
               super("pulsar_txn_tc", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   ```
   
   ```java
   private static class MLTxnPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTxnPendingAckLogBufferedWriterMetrics() {
               super("pulsar_txn_pending_ack_store", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   ```
   
   How are you feeling?



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   I have rebased `branch-master ` to solve the problem of Pulsar CI `broker 1`


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] tjiuming commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > > Hi @asafm
   > > > As I understand, this only happens once per instance of MLTransactionLogImpl. So I don't understand why you want to pass metric by metric. Something doesn't add up here.
   > > > I guess my main question is: how many MLTransactionLogImpl are there in a single broker process? More than 1?
   > > 
   > > 
   > > Yes, maybe more than one.
   > 
   > Ok, if more than one, then the design must change. I thought the whole idea was that you have a single instance of metrics per metric prefix. If not, after much thought I suggest the following:
   > 
   > ```java
   > abstract class BufferMetrics {
   > 	protected abstract void observeRecordsPerBatch(int)
   > 	protected abstract void incFlushTriggeredByMaxRecords(int)
   > }
   > 
   > MLTransactionMetadataStoreBufferedWriterMetrics extends BufferMetrics {
   > 	static private Histogram recordsPerBatchMetric = new Histogram.Builder()
   >                         .name("pulsar_tx_store_bufferedwriter_batch_record_count")
   >                         .labelNames(new String[]{"txCoordinatorId"})
   >                         .help("Records per batch histogram")
   >                         .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
   >                         .register(registry));
   > 	
   >     private Histogram.Child recordsPerBatchHistogram;                      
   > 
   > 
   > 	public MLTransactionMetadataStoreBufferedWriterMetrics(String txCoordinatorId) {
   > 	    recordsPerBatchHistogram = recordsPerBatchHistogram.labels(txCoordinatorId)
   > 
   > 	}
   >      
   >        protected observeRecordsPerBatch(value) {
   >             recordsPerBatchHistogram.observe(value)       
   >        }
   > 
   > }
   > ```
   > 
   > The pros:
   > 
   > * It's explicit
   > * No confusing pass of label names multiple times which after 2nd time are not really used.
   > 
   > The cons:
   > 
   > * A bit awkward
   > 
   > Another approach which I disliked a bit, but it's still ok: Add to Pulsar Common:
   > 
   > ```java
   > class PrometheusRegistryChecker {
   >      static defaultMetricRegistryNameToCollector = new HashMap<String, Collector>()
   > 
   >      static Collector registerIfNotExists(collector) {}
   > }
   > ```
   > 
   > Like `FunctionCollectorRegistryImpl`
   
   agreed, easier is better.


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/

Review Comment:
   Good idea. The name of the variable has been changed to`metircs` and removed this code comment.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/

Review Comment:
   Good idea. The name of the variable has been changed to`metrics` and removed this code comment.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);

Review Comment:
   I have renamed this metrics name suffix, thanks for suggestions :)



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);

Review Comment:
   I have renamed this metrics name suffix, thanks for suggestions :)



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   > There is a way to solve this issue by making sure we're not defining metrics twice:
   We know that we plan to create only one Metrics instance per metic-prefix. So in that case, both TxLog and PendingAckStoreProvider will create one with its own prefix, and that's it. No need to verify it was created before. In the event a future developer will make a mistake, it will fail in the constructor in some test right since CollectorRegistry.register() will fail on a duplicate.
   
   That's a good way to do it, but this can make the code confuse:
   
   - When the Transaction Log Provider opens a Transaction Log, passed the `Histogram` to Transaction Log. That is OK.
   - When the Transaction Log close, remove the labels. That is ok too.
   
   But the Transaction Log Provider will hold all the `Collector` of Txn Buffered Writer, this is confusing


-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Maybe `txnLogRecordCountPerBatchEntry`?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {
+                    metricsStats.triggerFlushByForce(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
                 doFlush();
                 return;
             }
             if (byScheduleThreads) {
+                if (metricsStats != null) {
+                    metricsStats.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
                 doFlush();
                 return;
             }
             AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
             if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
+                if (metricsStats != null) {
+                    metricsStats.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);

Review Comment:
   `flushContext.asyncAddArgsList.get(0)` is `firstAsyncAddArgs`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;

Review Comment:
   Maybe `batchSizeBytesMetric`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")

Review Comment:
   Records per batch histogram



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);

Review Comment:
   `bufferedwriter_flush_trigger_max_records`
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -144,8 +154,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
         this.flushContext = FlushContext.newInstance();
         this.dataArray = new ArrayList<>();
         this.state = State.OPEN;
-        this.timer = timer;
+        // Metrics.

Review Comment:
   No need for this comment as the code explains itself.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -89,15 +89,15 @@
     /**
      * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
      */
-    private final int batchedWriteMaxRecords;
+    final int batchedWriteMaxRecords;

Review Comment:
   What's the motivation for the removal of `private` from these three variables?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;

Review Comment:
   Maybe `batchSizeBytes`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   I would drop the prefix pulsar from the variable name and only keep it in the metric creation



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByRecords;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    Counter pulsarBatchedLogTriggeringCountBySize;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByDelayTime;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,

Review Comment:
   What do you mean by that explanation? It looks very much the same explanation of `pulsarBatchedLogTriggeringCountBySize`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -89,15 +89,15 @@
     /**
      * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
      */
-    private final int batchedWriteMaxRecords;
+    final int batchedWriteMaxRecords;

Review Comment:
   Why remove `private`?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/
+    private final TxnLogBufferedWriterMetricsStats metricsStats;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor, Timer timer,

Review Comment:
   Can you explain the meaning of instantiating a buffered writer without metrics?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -144,8 +154,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
         this.flushContext = FlushContext.newInstance();
         this.dataArray = new ArrayList<>();
         this.state = State.OPEN;
-        this.timer = timer;
+        // Metrics.
+        this.metricsStats = metricsStats;
         // scheduler task.
+        this.timer = timer;

Review Comment:
   You need to move the comment in line 159 back to its original place :)



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/

Review Comment:
   No need for that comment - it's obvious its metrics from the variable name `metricStats`. Actually, I would just name it `TxnLogBufferedWriterMetrics metrics;`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   Not sure I understand. If you write a single record which makes it be over max batch size, then you still write this as a batch with a single entry inside?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -212,6 +224,12 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
                 doTrigFlush(true, false);
             }
             ByteBuf byteBuf = dataSerializer.serialize(data);
+            /**

Review Comment:
   Why the comment is located here? Is this the place you force flush due to record size > batch size?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;

Review Comment:
   Maybe `recordsPerBatchHistogram`?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByRecords;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    Counter pulsarBatchedLogTriggeringCountBySize;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByDelayTime;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    Counter pulsarBatchedLogTriggeringCountByForce;
+
+    public static synchronized TxnLogBufferedWriterMetricsStats getInstance(
+                                                            TxnLogBufferedWriterMetricsDefinition metricsDefinition){
+        // Mark who references stats.
+        METRICS_INSTANCE_REFERENCE.computeIfAbsent(metricsDefinition.getComponent(), n -> new ArrayList<>());
+        METRICS_INSTANCE_REFERENCE.get(metricsDefinition.getComponent()).add(metricsDefinition);
+        // Get or create stats.
+        return METRICS_REGISTRY.computeIfAbsent(

Review Comment:
   you assume the same component has exactly the same label names since you only check existence by component name, yet the caller may call you up with different label names per component, you may end up returning a stats object with different lables



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;

Review Comment:
   Maybe `batchSizeBytesHistogram`



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -60,8 +61,14 @@
 import org.testng.annotations.Test;
 
 @Slf4j
+@Test(groups = "broker")

Review Comment:
   General comment: The test below don't check each case of trigger.
   
   I would expect each scenario to check the metrics to the correct ones. I wouldn't make up a test method to check the metrics which half of them are not really relevant. 
   
   If you have a scenario for batchRecordCount > max records flush, then check metric there, no?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;

Review Comment:
   `pulsarBatchedLogOldestRecordDelayTimeSeconds` --> `oldestRecordInBatchDelayTimeSecondsMetric`?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Maybe `pulsarBatchedLogRecordsCountPerEntry` --> `recordsPerBatchMetric`
   



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +620,322 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends JsonDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * 1. Verify Transaction buffered writer stats correct when enabled batch feature. Exclusive "triggerByForceFlush",
+     *    this property verified by {@link #testMetricsStatsWhenForceFlush()}.
+     * 2. Verify metrics will be release after {@link TxnLogBufferedWriterMetricsStats#clone()}.
+     */
+    @Test
+    public void testMetricsStatsWhenEnabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger to get the count of refresh event.
+        AtomicInteger refreshCount = new AtomicInteger();
+        String managedLedgerName = "-";
+        ManagedLedger managedLedger = Mockito.mock(ManagedLedger.class);

Review Comment:
   In general it's impossible to read this method and understand easily what you want to test, since it's filled with lots of plumbing.
   
   Example: You want to count how many times the managed ledger has been written to right? Awesome - create in a different method and return a data structure containing that managed ledger and counter - 2-3 lines tops.
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;

Review Comment:
   `pulsarBatchedLogTriggeringCountByRecords` --> `batchFlushTriggerdByMaxRecordsMetric`?



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +620,322 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends JsonDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * 1. Verify Transaction buffered writer stats correct when enabled batch feature. Exclusive "triggerByForceFlush",
+     *    this property verified by {@link #testMetricsStatsWhenForceFlush()}.
+     * 2. Verify metrics will be release after {@link TxnLogBufferedWriterMetricsStats#clone()}.
+     */
+    @Test
+    public void testMetricsStatsWhenEnabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger to get the count of refresh event.
+        AtomicInteger refreshCount = new AtomicInteger();
+        String managedLedgerName = "-";
+        ManagedLedger managedLedger = Mockito.mock(ManagedLedger.class);
+        Mockito.when(managedLedger.getName()).thenReturn(managedLedgerName);
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                refreshCount.incrementAndGet();
+                AsyncCallbacks.AddEntryCallback callback =
+                        (AsyncCallbacks.AddEntryCallback) invocation.getArguments()[1];
+                callback.addComplete(PositionImpl.get(1,1), (ByteBuf)invocation.getArguments()[0],
+                        invocation.getArguments()[2]);
+                return null;
+            }
+        }).when(managedLedger).asyncAddEntry(Mockito.any(ByteBuf.class), Mockito.any(), Mockito.any());
+        // Mock addDataCallbackCount to get the count of add data finish count;
+        AtomicInteger addDataCallbackFinishCount = new AtomicInteger();
+        AtomicInteger addDataCallbackFailureCount = new AtomicInteger();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object context) {
+                addDataCallbackFinishCount.incrementAndGet();
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+                addDataCallbackFailureCount.incrementAndGet();
+            }
+        };
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, true, metricsStats);
+        // Add some data.
+        int writeCount = 3000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        Assert.assertEquals(getCounterValue(String.format("%s_batched_log_triggering_count_by_force", metricsPrefix)), 0D);
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_records", metricsPrefix))
+                + getCounterValue(String.format("%s_batched_log_triggering_count_by_size", metricsPrefix))
+                + getCounterValue(String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix)),
+                (double)refreshCount.get());
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                refreshCount.get());
+        Assert.assertEquals(
+                getHistogramSum(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                writeCount);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                refreshCount.get());
+        Assert.assertEquals(
+                getHistogramSum(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                dataSerializer.getTotalSize());
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix)),
+                refreshCount.get());
+        /**
+         * Assert all metrics will be released after {@link TxnLogBufferedWriter#close()}
+         *   1. Register another {@link TxnLogBufferedWriter}
+         *   2. Close first {@link TxnLogBufferedWriter}, verify the labels of metrics will be released after
+         *      {@link TxnLogBufferedWriter#close()}
+         *   3. Close second {@link TxnLogBufferedWriter},verify all metrics will be released after all
+         *      {@link TxnLogBufferedWriter#close()}
+         */
+        TxnLogBufferedWriterMetricsStats anotherMetricsStat = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, new String[]{"2"}, CollectorRegistry.defaultRegistry);
+        TxnLogBufferedWriter<Integer> anotherTxnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, true, anotherMetricsStat);
+        anotherTxnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount + 1
+        );
+        // Close first-writer, verify the labels will be released after writer-close.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> getHistogramCount(
+                        String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix)) == 0
+        );
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_records", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_size", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                0D
+        );
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                0D
+        );
+        // Close second-writer, verify all metrics will be released after all writer-close.
+        anotherTxnLogBufferedWriter.close();
+        anotherMetricsStat.close();
+        // cleanup.
+        transactionTimer.stop();
+        managedLedger.close();
+        orderedExecutor.shutdown();
+    }
+
+    private double getCounterValue(String name) {
+        Double d = CollectorRegistry.defaultRegistry.getSampleValue(
+                name + "_total",
+                metricsLabelNames,
+                metricsLabelValues);
+        return d == null ? 0: d.doubleValue();
+    }
+
+    private double getHistogramCount(String name) {
+        Double d = CollectorRegistry.defaultRegistry.getSampleValue(
+                name + "_count",
+                metricsLabelNames,
+                metricsLabelValues);
+        return d == null ? 0: d.doubleValue();
+    }
+
+    private double getHistogramSum(String name) {
+        Double d = CollectorRegistry.defaultRegistry.getSampleValue(
+                name + "_sum",
+                metricsLabelNames,
+                metricsLabelValues);
+        return d == null ? 0: d.doubleValue();
+    }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Mock addDataCallbackCount to get the count of add data finish count;
+        AtomicInteger addDataCallbackFinishCount = new AtomicInteger();
+        AtomicInteger addDataCallbackFailureCount = new AtomicInteger();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object context) {
+                addDataCallbackFinishCount.incrementAndGet();
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+                addDataCallbackFailureCount.incrementAndGet();
+            }
+        };
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        Assert.assertEquals(getCounterValue(String.format("%s_batched_log_triggering_count_by_force", metricsPrefix)), 0D);
+        Assert.assertEquals(
+                getCounterValue(String.format("%s_batched_log_triggering_count_by_records", metricsPrefix))
+                    + getCounterValue(String.format("%s_batched_log_triggering_count_by_size", metricsPrefix))
+                    + getCounterValue(String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix)),
+                0D);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_records_count_per_entry", metricsPrefix)),
+                0D);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_entry_size_bytes", metricsPrefix)),
+                0D);
+        Assert.assertEquals(
+                getHistogramCount(String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix)),
+                0D);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * Test {@link TxnLogBufferedWriterMetricsStats#triggerFlushByForce(int, long, long)}.
+     */
+    @Test
+    public void testMetricsStatsWhenForceFlush() throws Exception {

Review Comment:
   If you remove public force flush then this test can be removed?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;

Review Comment:
   Extra space



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   There is a way to solve this issue by making sure we're not defining metrics twice:
   We know that we plan to create only one Metrics instance per metic-prefix. So in that case, both TxLog and PendingAckStoreProvider will create one with its own prefix, and that's it. No need to verify it was created before. In the event a future developer will make a mistake, it will fail in the constructor in some test right since CollectorRegistry.register() will fail on a duplicate.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);

Review Comment:
   `%s_batched_log_records_count_per_entry`
   
   say
   `pulsar_txn_pending_ack`
   then
   
   `pulsar_txn_pending_ack_bufferedwriter_batch_record_count`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;

Review Comment:
   You get the drift



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")

Review Comment:
   Max record latency in batch hisogram
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);

Review Comment:
   `_bufferedwriter_batch_oldest_record_delay_time_second`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")

Review Comment:
   Batch size in bytes histogram



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByRecords = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByRecordsName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByRecordsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxRecords\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByRecordsChild = pulsarBatchedLogTriggeringCountByRecords.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountBySizeName =
+                String.format("%s_batched_log_triggering_count_by_size", metricsPrefix);
+        pulsarBatchedLogTriggeringCountBySize = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountBySizeName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountBySizeName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold \"batchedWriteMaxSize\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountBySizeChild = pulsarBatchedLogTriggeringCountBySize.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByDelayTimeName =
+                String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByDelayTime = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByDelayTimeName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByDelayTimeName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxDelayInMillis\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByDelayTimeChild =
+                pulsarBatchedLogTriggeringCountByDelayTime.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByForcename =

Review Comment:
   This will change after your PR fix



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +620,322 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends JsonDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * 1. Verify Transaction buffered writer stats correct when enabled batch feature. Exclusive "triggerByForceFlush",
+     *    this property verified by {@link #testMetricsStatsWhenForceFlush()}.
+     * 2. Verify metrics will be release after {@link TxnLogBufferedWriterMetricsStats#clone()}.
+     */
+    @Test
+    public void testMetricsStatsWhenEnabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger to get the count of refresh event.
+        AtomicInteger refreshCount = new AtomicInteger();

Review Comment:
   What do you mean by counting refresh? I see in buffer that managed ledger asyncAddEntry is called mainly when batch is flush, so why not call it batchFlushCount?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByRecords = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByRecordsName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByRecordsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxRecords\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByRecordsChild = pulsarBatchedLogTriggeringCountByRecords.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountBySizeName =
+                String.format("%s_batched_log_triggering_count_by_size", metricsPrefix);
+        pulsarBatchedLogTriggeringCountBySize = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountBySizeName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountBySizeName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold \"batchedWriteMaxSize\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountBySizeChild = pulsarBatchedLogTriggeringCountBySize.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByDelayTimeName =

Review Comment:
   `bufferedwriter_flush_trigger_max_delay`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByRecords = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByRecordsName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByRecordsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxRecords\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByRecordsChild = pulsarBatchedLogTriggeringCountByRecords.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountBySizeName =
+                String.format("%s_batched_log_triggering_count_by_size", metricsPrefix);

Review Comment:
   `bufferedwriter_flush_trigger_max_size`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.

Review Comment:
   Good idea, already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.

Review Comment:
   Already removed this comment



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Hi @asafm 
   
   > Can you explain this further? I didn't understand how the provider would hold all the Collectors (metrics). It should only be to a single instance of the metric class.
   
   I responded separately below



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   
   > > But the Transaction Log Provider will hold all the Collector of Txn Buffered Writer; this is confusing
   
   > Can you explain this further? I didn't understand how the provider would hold all the Collectors (metrics). It should only be to a single instance of the metric class.
   
   When `MLTransactionMetadataStoreProvider` initialized, we create `Collector` like this: 
   
   ```java
   public MLTransactionMetadataStoreProvider(){
      this.recordsPerBatchMetric = ...
      this.batchSizeBytesMetric = ...
      this.oldestRecordInBatchDelayTimeSecondsMetric = ...
      this.batchFlushTriggeredByMaxRecordsMetric = ...
      this.batchFlushTriggeredByMaxSizeMetric = ...
      this.batchFlushTriggeredByMaxDelayMetric = ...
   }
   ```
   
   And when creating `MlTransactionLogImpl`, pass these `Collector` to `MlTransactionLogImpl` like this: 
   
   ```java
   public class MLTransactionMetadataStoreProvider{
     public TransactionMetadataStore openStore(...){
       TransactionMetadataStore store = ...;
       setMetrics(store);
       return store;
     }
   
     private void setMetrics(TransactionMetadataStore store) {
       store.recordsPerBatchMetric = this.recordsPerBatchMetric;
       store.batchSizeBytesMetric = this.batchSizeBytesMetric;
       store.oldestRecordInBatchDelayTimeSecondsMetric = this.oldestRecordInBatchDelayTimeSecondsMetric;
       ...
     }
   }
   ```
   
   The `MLTransactionMetadataStoreProvider` will hold all the Collector of Txn Buffered Writer, this is confusing


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > Ok. So the if metrics != null will go away as well, right?
   
   Yes, everything works, except metrics


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Yes, maybe more than 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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);

Review Comment:
   That's not entirely true. Look at the implementation
   ```java
       private static ByteBuf wrappedUnmodifiableBuffer(boolean copy, ByteBuf... buffers) {
           switch (buffers.length) {
           case 0:
               return EMPTY_BUFFER;
           case 1:
               return buffers[0].asReadOnly();
           default:
               if (copy) {
                   buffers = Arrays.copyOf(buffers, buffers.length, ByteBuf[].class);
               }
               return new FixedCompositeByteBuf(ALLOC, buffers);
           }
       }
   ```
   
   So you do get a composite bytebuf, because you want to avoid the copy of course.
   
   When you're not using the pulsar version you're missing out on several configurations that might come in handy.
   @codelipenghui WDYT?
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   > Does it make sense to log the error instead of calling callback.addFailed?
   
   It simply prints all the exceptions that have occurred in asynchronous tasks and are used to locate problems.
   
   As long as the request is in the memory queue, it will eventually succeed or fail, we do not need redundant executive `callback.addFailed`



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   Can't you just create your own single thread executor service which is ScheduledExecutorService and use it for the scheduling of period flush instead of implementing it on your own using Timer?
   



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        verifyTheCounterMetrics(0,0, actualBatchFlushCount,0);
+        verifyTheHistogramMetrics(actualBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByLargeSingleData() throws Exception {
+        // Use TwoLenSumDataSerializer for: write a little data once, then write a large data once.
+        int bytesSizePerRecordWhichInBatch = 4;
+        int batchedWriteMaxSize = 1024;
+        TwoLenSumDataSerializer dataSerializer =
+                new TwoLenSumDataSerializer(bytesSizePerRecordWhichInBatch, batchedWriteMaxSize);
+        int writeCount = 100;
+        int singleLargeDataRequestCount = writeCount / 2;
+        int expectedBatchFlushTriggeredByLargeData = singleLargeDataRequestCount;
+        int expectedTotalBytesSize = expectedBatchFlushTriggeredByLargeData * bytesSizePerRecordWhichInBatch;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, i);
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(

Review Comment:
   You need to check :
   >Where are you checking that you wrote to managed ledger writeCount times?



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){

Review Comment:
   No, `dataArray` is only modified in these two methods `doFlush` and `internalAsyncAddData`, these methods only running in a single thread `singleThreadExecutorForWrite`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,7 +143,7 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
-        this.batchEnabled = batchEnabled;
+        this.batchEnabled = batchEnabled && batchedWriteMaxRecords > 1;

Review Comment:
   Good idea, already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,230 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counter as other trigger types.

Review Comment:
   Sorry, Already fixed.



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   
   Hi @asafm
   
   I've taken care of all the comments, could you review this PR again?
   
   


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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Ok. I can leave with that but just to be on the safe side I asked also for @tjiuming advice here. 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,230 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counter as other trigger types.

Review Comment:
   I know I wrote it but I `counter` --> `counted`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){
+                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);

Review Comment:
   1. import static ManagedLedgerException to shorten expression.
   2. Can you please explain why did you choose the Fenced exception for this specific error which you don't know its nature?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){

Review Comment:
   `dataArray` is subject to modification by other threads: Other calling threads might add records to it, while you're inside `internalAsyncAddData()`, so you can't really the count to be equal, right?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);

Review Comment:
   We need to close this with @codelipenghui 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -546,10 +552,9 @@ public void addCallback(AddDataCallback callback, Object ctx){
         }
     }
 
-    /** Callback for batch write BK. **/
-    private final BufferedAddEntryCallback bufferedAddEntryCallback = new BufferedAddEntryCallback();
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();
 
-    private class BufferedAddEntryCallback implements AsyncCallbacks.AddEntryCallback{
+    private class BookKeeperBatchedWriteCallback implements AsyncCallbacks.AddEntryCallback{

Review Comment:
   if your callback doesn't need to access any variables located in `TxnLogBufferedWriter` then perhaps it's better to declare it `private static class`
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -541,8 +545,52 @@ public void recycle(){
             this.asyncAddArgsList.clear();
             this.handle.recycle(this);
         }
+
+        public void addCallback(AddDataCallback callback, Object ctx){
+            AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+            asyncAddArgsList.add(asyncAddArgs);
+        }
     }
 
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();

Review Comment:
   As I wrote [here](https://github.com/apache/pulsar/pull/16758#discussion_r939988061), this line should be at the beginning of `TxnLogBufferedWriter` where all the variables are placed.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   Look:
   You have  threads:
   1. singleThreadExecutorForWrite which access `asyncAddArgList` (check if it's empty, iterates over it, ...). 
   2. `addComplete` and `addFailed` : both access `asyncAddArgList` but from the orderedExecutor threads which are inside `ManagedLedgerImpl`. 
   
   Thus you have concurrent access to  `asyncAddArgsList` which is not protected by concurrent access
            



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   Completly lost you here



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,7 +143,7 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
-        this.batchEnabled = batchEnabled;
+        this.batchEnabled = batchEnabled && batchedWriteMaxRecords > 1;

Review Comment:
   If you're disabling it due to misconfiguration, let the user know by a warning log line



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   But @poorbarcode , here is the javadoc of `scheduleWithFixedDelay`:
   >Creates and executes a periodic action that becomes enabled first after the given initial delay, and subsequently with the given delay between the termination of one execution and the commencement of the next. If any execution of the task encounters an exception, subsequent executions are suppressed. Otherwise, the task will only terminate via cancellation or termination of the executor.
   
   You can clearly see it will also avoid a large number of fixed-time flush tasks.
   In fact, it is *exactly* the functionality you wrote using Timer 
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);

Review Comment:
   Ok, you are correct. The name confused me. This FixedCompositeByteBuf allocates nothing: Just an instance that contains an array of bytebuf as one bytebuf
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it only used in the constructor.
   
   Yes, we need.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get it. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we need to get the `Collector` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like `collectorRegistry.getRegistedCollector(String name)`, and it will throws IllegalArgumentException when we registering collector with the same name more than once, see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it only used in the constructor.
   
   Yes, we need.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like `collectorRegistry.getRegistedCollector(String name)`, and it will throws IllegalArgumentException when we registering collector with the same name more than once, see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some code comments in this PR(It's now written in Modifications).
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR( plan 7-2 in Modifications ). This is compatible only with split PR, making each PR have less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -212,6 +224,12 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
                 doTrigFlush(true, false);
             }
             ByteBuf byteBuf = dataSerializer.serialize(data);
+            /**

Review Comment:
   I've rewritten the logic that triggers the flush, could you take a look at it again ?



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some comments here.
   To reduce the cost of maintenance later, I didn't want to support disabled metrics stat in the design, even if the Txn Buffered Writer disabled the batch feature. This constructor which without `metricsStats`, and these "null checks" will be removed in the next PR. This is only compatible to split the PR, making each PR has less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;

Review Comment:
   Good idea. Already named this variable name to`batchSizeBytesMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/
+    private final TxnLogBufferedWriterMetricsStats metricsStats;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor, Timer timer,

Review Comment:
   Yes, it is the section "Code will be removed in the next PR (7-2)" of Motivation



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,138 +353,65 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
-        // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteBuf, contentByteBuf);
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
-        }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
-    }
-
-    /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
-     */
-    @Override
-    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        try {
-            final int batchSize = flushContext.asyncAddArgsList.size();
-            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
-                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
-                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
-                        batchIndex);
-                // Because this task already running at ordered task, so just "run".
-                try {
-                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
-                } catch (Exception e){
-                    log.error("After writing to the transaction batched log complete, the callback failed."
-                            + " managedLedger: " + managedLedger.getName(), e);
-                }
-            }
-        } finally {
-            flushContext.recycle();
+            managedLedger.asyncAddEntry(wholeByteBuf, bookKeeperBatchedWriteCallback, flushContext);
         }
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();
+        bytesSize = 0;
     }
 
     /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
-     */
-    @Override
-    public void addFailed(ManagedLedgerException exception, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        failureCallbackByContextAndRecycle(flushContext, exception);
-    }
-
-    /**
-     * Cancel pending tasks and release resources.
+     * Release resources and cancel pending tasks.
      */
     @Override
     public void close() {
-        // If disabled batch feature, there is no closing state.
+        // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
             return;
         }
-        // Prevent the reentrant.
+        // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            // Other thread also calling "close()".
             return;
         }
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
-            if (state == State.CLOSED){
-                return;
-            }
-            // Failure callback to pending request.
-            // If some request has been flushed, Bookie triggers the callback.
-            failureCallbackByContextAndRecycle(this.flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
-            // Cancel task that schedule at fixed rate trig flush.
-            if (timeout == null){
-                log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                        + " is null. managedLedger: " + managedLedger.getName());
-            } else if (timeout.isCancelled()){
-                // TODO How decisions the timer-task has been finished ?
-                this.state = State.CLOSED;
-            } else {
-                if (this.timeout.cancel()) {
-                    this.state = State.CLOSED;
+            try {
+                if (state == State.CLOSED) {
+                    return;
+                }
+                // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
+                // fail.
+                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                // Cancel the timing task.
+                if (timeout == null) {
+                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
+                            + " is null. managedLedger: " + managedLedger.getName());
+                } else if (timeout.isCancelled()) {
+                    // TODO How decisions the timer-task has been finished ?
+                    STATE_UPDATER.set(this, State.CLOSED);
                 } else {
-                    // Cancel task failure, The state will stay at CLOSING.
-                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
-                            + " stay at CLOSING. managedLedger: " + managedLedger.getName());
+                    if (this.timeout.cancel()) {
+                        STATE_UPDATER.set(this, State.CLOSED);
+                    } else {
+                        // Cancel task failure, The state will stay at CLOSING.
+                        log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"

Review Comment:
   I have already rewrite the method `close`:
   
   - Just call `timeout.cancel` and set the state to `closed`, don't care if it's actually executed.
   - In method `nextTimingTrigger`, Prevents timing trigger.
   
   These two measures ensure that the Timing trigger will not be executed after `close`.
   
   ----
   
   And there are two behaviors that are not as expected( So the above plan is optimal ):
   
   - if the stat is EXPIRED, not means already executed, this just means `expired`(already executed or not). 
   - when we call `timeout.cancel`, will close the task asynchrony and return nothing.
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".

Review Comment:
   already removed these unnecessary comments.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            if (flushContext.asyncAddArgsList.isEmpty()) {
+                return;
+            }
+            if (metrics != null) {
+                metrics.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            // Start the next timing task.
+            nextTimingTrigger();
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
+            if (metrics != null) {
+                metrics.triggerFlushByRecordsCount(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize) {
+            if (metrics != null) {
+                metrics.triggerFlushByBytesSize(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+        }
+    }
+
+    /**

Review Comment:
   Already removed this comment.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            if (flushContext.asyncAddArgsList.isEmpty()) {
+                return;
+            }
+            if (metrics != null) {
+                metrics.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            // Start the next timing task.
+            nextTimingTrigger();
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   Already removed `this.` all places like 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] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > I still see the comment. `/** Metrics. **/`.
   
   Already removed.


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   Does it make sense to log the error instead of calling `callback.addFailed`? @codelipenghui WDYT?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.

Review Comment:
   This comment is completely redundant. 



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();

Review Comment:
   Where are you checking `actualBatchFlushCount == 1` ?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.

Review Comment:
   I don't understand: The comment says, "append data to the data array"
   and then the code says `dataArray.add(data)`, so the comment says the same thing as the code --> comment can be deleted



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);

Review Comment:
   maybe `prefixByteBuf`?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.

Review Comment:
   Ask yourself this please: Can I read the code and understand it without this comment? 
   If yes, delete the comment.
   If no:
      Can I improve the code so it can be understood without the comment?
         Yes?
              Improve code
         No?
             Write comment
   
   
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   Why `flushContext.asyncAddArgsList.size()` and not `dataArray.size()`? I mean, it's odd to check the number of callbacks instead of checking plainly the size of the data queue. If agree, I would replace below as well
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);

Review Comment:
   Why `Unpooled` and not `PulsarByteBufAllocator`?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   As I mentioned in the previous PR, it seems that you just implemented `ScheduledExecutorService.scheduleWithFixedDelay` by yourself using Timer.
   
   Any reason not to replace it with scheduleWithFixedDelay?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.

Review Comment:
   Maybe 
   ```java
   flushContext.addCallback(callback, ctx)
   ```
   
   instead of 
   ```java
   // Append callback to the flushContext.
   AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
           flushContext.asyncAddArgsList.add(asyncAddArgs);
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);

Review Comment:
   Here you chose not to notify the callbacks because you assume the queue is still with elements, next timer task will flush them? @codelipenghui WDYT?
   
   I'm comparing with this
   ```java
           if (State.CLOSING == state || State.CLOSED == state){
               failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.

Review Comment:
   You know what's the biggest problem with lengthy comments? When you refactor, you usually don't change the comments, so they stay out of sync with the code. 
   Look here, you called pairByteBuf, but not it's `wholeByteBuf` so the reader wouldn't really understand this comment.
   
   You're hiding the release of the buffer, created right here in this method, inside FlushContext.recycle() called from addComplete.
   
   I suggest you create new callback which calls the original callback but also releases the `wholeByteBuf` so I can see the code that releases this object right next it's creation.
   
   and the delete this comment, which is also *duplicated* in 
   ```java
           /**
            * If turning on the Batch feature, we need to release the byteBuf produced by
            * {@link DataSerializer#serialize(ArrayList)} when Managed ledger async add callback.
            * Only carry the ByteBuf objects, no other use.
            */
   ```



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,

Review Comment:
   I can't guess what is 1, which is the most important number here. Let's name it: maxBatchDelaySeconds



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
+            managedLedger.asyncAddEntry(wholeByteBuf, this, flushContext);
         }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
+        // Reset the cache.

Review Comment:
   Redundant comment



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   IMO this explanation is confusing and doesn't serve what you truly want to convey. 
   I would add the following explanation:
   
   Note-2: A batch has numerous triggers. The metrics in this class count each type of trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but the big record itself is not written in batch hence is not included in the batch metrics written above (batch size, batch delay, etc). The trigger is of course counter as other trigger types.
   
   
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
+            managedLedger.asyncAddEntry(wholeByteBuf, this, flushContext);
         }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
+        // Reset the cache.
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();

Review Comment:
   I have to say something. 
   At first, when I saw this line, I was sure there was a bug. How can it be that you are creating a new flush context without recycling the existing flush context - you are overwriting the reference - you're going to lose the reference, it must be a bug!
   
   So I started reading the code. It took me quite some time to figure out `this` in `managedLedger.asyncAddEntry()` a couple of lines above it, is actually a reference to a callback. Why the buffered writer also acts as a callback? Why not create a callback class and reference it here, so it will be obvious? where to search.
   
   



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        verifyTheCounterMetrics(0,0, actualBatchFlushCount,0);
+        verifyTheHistogramMetrics(actualBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByLargeSingleData() throws Exception {
+        // Use TwoLenSumDataSerializer for: write a little data once, then write a large data once.
+        int bytesSizePerRecordWhichInBatch = 4;
+        int batchedWriteMaxSize = 1024;
+        TwoLenSumDataSerializer dataSerializer =
+                new TwoLenSumDataSerializer(bytesSizePerRecordWhichInBatch, batchedWriteMaxSize);
+        int writeCount = 100;
+        int singleLargeDataRequestCount = writeCount / 2;
+        int expectedBatchFlushTriggeredByLargeData = singleLargeDataRequestCount;
+        int expectedTotalBytesSize = expectedBatchFlushTriggeredByLargeData * bytesSizePerRecordWhichInBatch;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, i);
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(

Review Comment:
   Where are you checking that you wrote to managed ledger writeCount times?
   
   I think this test is hard to understand.
   
   I would do:
   ```
       createBufferedWriter(maxBatchSize = batchedWriteMaxSize, maxRecords = MAX_INT, maxDelay = MAX_INT)
       writeToBuffer(randomRecord(size = batchedWriteMaxSize/10)
       writeToBuffer(randomRecord(size = batchedWriteMaxSize/10)
       writeToBuffer(randomRecord(size = maxBatchSize)
       waitForAllWriteToCompleteSuccessfully()
       assertThat(actualNumOfFlushes, 1)
       assertThat(actualWritesToManagedLedger,2)
   ```
       



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/
+    private final TxnLogBufferedWriterMetricsStats metricsStats;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor, Timer timer,

Review Comment:
   Your comment 
   >Yes, everything works, except metrics
   
   I asked if you will remove the `if metrics != null` statements in the next PRs.



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);

Review Comment:
   Why do you need this sleep at all?
   Just write 100 records, and you wait for the 1 second timeout to pass



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Either what you wrote is completely the wrong direction or I completely misunderstood you :)
   
   I thought that in MLTransactionLogImp, when open a ledger you also create a buffered writer, thus in here you will also pass an instance of `TxnLogBufferedWriterMetricsStats`.
   
   ```java
                       public void openLedgerComplete(ManagedLedger ledger, Object ctx) {
                           MLTransactionLogImpl.this.managedLedger = ledger;
                           MLTransactionLogImpl.this.bufferedWriter = new TxnLogBufferedWriter<>(
                                   managedLedger, ((ManagedLedgerImpl) managedLedger).getExecutor(),
                                   timer, TransactionLogDataSerializer.INSTANCE,
                                   txnLogBufferedWriterConfig.getBatchedWriteMaxRecords(),
                                   txnLogBufferedWriterConfig.getBatchedWriteMaxSize(),
                                   txnLogBufferedWriterConfig.getBatchedWriteMaxDelayInMillis(),
                                   txnLogBufferedWriterConfig.isBatchEnabled());
   ```
   
   As I understand, this only happens once per instance of `MLTransactionLogImpl`. So I don't understand why you want to pass metric by metric. Something doesn't add up here. 
   
   I guess my main question is: how many  `MLTransactionLogImpl` are there in a single broker process? More than 1?
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        verifyTheCounterMetrics(0,0, actualBatchFlushCount,0);
+        verifyTheHistogramMetrics(actualBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByLargeSingleData() throws Exception {
+        // Use TwoLenSumDataSerializer for: write a little data once, then write a large data once.
+        int bytesSizePerRecordWhichInBatch = 4;
+        int batchedWriteMaxSize = 1024;
+        TwoLenSumDataSerializer dataSerializer =
+                new TwoLenSumDataSerializer(bytesSizePerRecordWhichInBatch, batchedWriteMaxSize);
+        int writeCount = 100;
+        int singleLargeDataRequestCount = writeCount / 2;
+        int expectedBatchFlushTriggeredByLargeData = singleLargeDataRequestCount;
+        int expectedTotalBytesSize = expectedBatchFlushTriggeredByLargeData * bytesSizePerRecordWhichInBatch;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, i);
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(

Review Comment:
   It seems to make no difference. In this unit test, we use `TwoLenSumDataSerializer` to control the size of each write: [4, batchedWriteMaxSize, batchedWriteMaxSize, 4, batchedWriteMaxSize....]



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.

Review Comment:
   Already delete this comment, thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.

Review Comment:
   Good idea, already fixed



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

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

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


[GitHub] [pulsar] asafm commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > Hi @asafm
   > 
   > > As I understand, this only happens once per instance of MLTransactionLogImpl. So I don't understand why you want to pass metric by metric. Something doesn't add up here.
   > > I guess my main question is: how many MLTransactionLogImpl are there in a single broker process? More than 1?
   > 
   > Yes, maybe more than one.
   
   Ok, if more than one, then the design must change. I thought the whole idea was that you have a single instance of metrics per metric prefix.
   If not, after much thought I suggest the following:
   ```java
   abstract class BufferMetrics {
   	protected abstract void observeRecordsPerBatch(int)
   	protected abstract void incFlushTriggeredByMaxRecords(int)
   }
   
   MLTransactionMetadataStoreBufferedWriterMetrics extends BufferMetrics {
   	static private Histogram recordsPerBatchMetric = new Histogram.Builder()
                           .name("pulsar_tx_store_bufferedwriter_batch_record_count")
                           .labelNames(new String[]{"txCoordinatorId"})
                           .help("Records per batch histogram")
                           .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
                           .register(registry));
   	
       private Histogram.Child recordsPerBatchHistogram;                      
   
   
   	public MLTransactionMetadataStoreBufferedWriterMetrics(String txCoordinatorId) {
   	    recordsPerBatchHistogram = recordsPerBatchHistogram.labels(txCoordinatorId)
   
   	}
   }
   ```
   
   The pros:
   * It's explicit
   * No confusing pass of label names multiple times which after 2nd time are not really used.
   
   The cons:
   * A bit awkward
   
   Another approach which I disliked a bit, but it's still ok:
   Add to Pulsar Common:
   ```java
   class PrometheusRegistryChecker {
        static defaultMetricRegistryNameToCollector = new HashMap<String, Collector>()
   
        static Collector registerIfNotExists(collector) {}
   }
   ```
   Like `FunctionCollectorRegistryImpl`
   
   


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   That's a good way to do it, but this can make the code confuse:
   
   - When the Transaction Log Provider opens a Transaction Log, passed the `Histogram` to Transaction Log. That is OK.
   - When the Transaction Log close, remove the labels. That is ok too.
   
   But the Transaction Log Provider will hold all the `Collector` of Txn Buffered Writer, this is confusing



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Hi @asafm 
   
   That's a good way to do it, but this can make the code confuse:
   
   - When the Transaction Log Provider opens a Transaction Log, passed the `Histogram` to Transaction Log. That is OK.
   - When the Transaction Log close, remove the labels. That is ok too.
   
   But the Transaction Log Provider will hold all the `Collector` of Txn Buffered Writer, this is confusing



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        verifyTheCounterMetrics(writeCount / 2,0,0,0);
+        verifyTheHistogramMetrics(batchFlushCounter.get(), writeCount, writeCount * 4);

Review Comment:
   > Do this call for each you verify
   
   Correct.  I've used a better way: calculate `expectedBacthFlushCount` and verify it equals to `batchFlushCounter.get()`, then replace these expressions with meaningful variable names.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea. The name of the variable has been changed to `recordsPerBatchMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea. Already named this variable name to `recordsPerBatchMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea. Already drop these prefix



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByRecords = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByRecordsName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByRecordsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxRecords\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByRecordsChild = pulsarBatchedLogTriggeringCountByRecords.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountBySizeName =
+                String.format("%s_batched_log_triggering_count_by_size", metricsPrefix);
+        pulsarBatchedLogTriggeringCountBySize = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountBySizeName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountBySizeName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold \"batchedWriteMaxSize\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountBySizeChild = pulsarBatchedLogTriggeringCountBySize.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByDelayTimeName =

Review Comment:
   I have renamed this metrics name suffix, thanks for suggestions :)



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByRecords = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByRecordsName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByRecordsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxRecords\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByRecordsChild = pulsarBatchedLogTriggeringCountByRecords.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountBySizeName =
+                String.format("%s_batched_log_triggering_count_by_size", metricsPrefix);

Review Comment:
   I have renamed this metrics name suffix, thanks for suggestions :)



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);

Review Comment:
   I have renamed this metrics name suffix, thanks for suggestions :)



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){
+                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);

Review Comment:
   > import static ManagedLedgerException to shorten expression.
   
   The ManagedLedgerException created here uses the parameter `e`, so it cannot be set as a static variable.
   
   > Can you please explain why did you choose the Fenced exception for this specific error which you don't know its nature?
   
   No exception is thrown in the block of "try catch". If it does, it will be due to coding errors such as `data Serializer` or `metrics`.



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

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

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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){
+                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);

Review Comment:
   I agree, we should not throw ManagedLedgerException out of ManagedLedger.
   IMO, we should use `PersistenceException` instead. 



-- 
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] asafm commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > @tjiuming @poorbarcode I actually found this pattern already in Pulsar. See `ComponentStatsManager`
   
   Why - this was only given as an example. Not relevant now since you already modified the original code.


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,138 +353,65 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
-        // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteBuf, contentByteBuf);
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
-        }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
-    }
-
-    /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
-     */
-    @Override
-    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        try {
-            final int batchSize = flushContext.asyncAddArgsList.size();
-            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
-                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
-                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
-                        batchIndex);
-                // Because this task already running at ordered task, so just "run".
-                try {
-                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
-                } catch (Exception e){
-                    log.error("After writing to the transaction batched log complete, the callback failed."
-                            + " managedLedger: " + managedLedger.getName(), e);
-                }
-            }
-        } finally {
-            flushContext.recycle();
+            managedLedger.asyncAddEntry(wholeByteBuf, bookKeeperBatchedWriteCallback, flushContext);
         }
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();
+        bytesSize = 0;
     }
 
     /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
-     */
-    @Override
-    public void addFailed(ManagedLedgerException exception, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        failureCallbackByContextAndRecycle(flushContext, exception);
-    }
-
-    /**
-     * Cancel pending tasks and release resources.
+     * Release resources and cancel pending tasks.
      */
     @Override
     public void close() {
-        // If disabled batch feature, there is no closing state.
+        // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
             return;
         }
-        // Prevent the reentrant.
+        // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            // Other thread also calling "close()".
             return;
         }
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
-            if (state == State.CLOSED){
-                return;
-            }
-            // Failure callback to pending request.
-            // If some request has been flushed, Bookie triggers the callback.
-            failureCallbackByContextAndRecycle(this.flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
-            // Cancel task that schedule at fixed rate trig flush.
-            if (timeout == null){
-                log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                        + " is null. managedLedger: " + managedLedger.getName());
-            } else if (timeout.isCancelled()){
-                // TODO How decisions the timer-task has been finished ?
-                this.state = State.CLOSED;
-            } else {
-                if (this.timeout.cancel()) {
-                    this.state = State.CLOSED;
+            try {
+                if (state == State.CLOSED) {
+                    return;
+                }
+                // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
+                // fail.
+                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                // Cancel the timing task.
+                if (timeout == null) {
+                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"

Review Comment:
   Already deleted this check



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/validator/TransactionBatchedWriteValidator.java:
##########
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.validator;
+
+import org.apache.pulsar.broker.ServiceConfiguration;
+
+public class TransactionBatchedWriteValidator {
+
+    public static void validate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Configuration field "
+                        + "'transactionPendingAckBatchedWriteMaxRecords' value must be greater than 10");

Review Comment:
   Already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){

Review Comment:
   Already fixed



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Txn pending ack batched write max records suggestion at least 10");
+            }
+            if (configuration.getTransactionPendingAckBatchedWriteMaxSize() < 1024 * 128){

Review Comment:
   Already fixed



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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/validator/TransactionBatchedWriteValidator.java:
##########
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.validator;
+
+import org.apache.pulsar.broker.ServiceConfiguration;
+
+public class TransactionBatchedWriteValidator {
+
+    public static void validate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Configuration field "
+                        + "'transactionPendingAckBatchedWriteMaxRecords' value must be greater than 10");

Review Comment:
   >=10? Sorry missed it the first time, must be greater than or equal to 10.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -239,33 +239,35 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
      */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         // Avoid missing callback, do failed callback when error occur before add data to the array.
-        boolean shouldCompensateCallBackWhenWriteFail = true;
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        int dataLength;
         try {
-            if (state == State.CLOSING || state == State.CLOSED){
-                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
-                return;
-            }
-            int dataLength = dataSerializer.getSerializedSize(data);
-            if (dataLength >= batchedWriteMaxSize){
-                trigFlushByLargeSingleData();
-                ByteBuf byteBuf = dataSerializer.serialize(data);
-                shouldCompensateCallBackWhenWriteFail = false;
-                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
-                return;
-            }
-            dataArray.add(data);
+            dataLength = dataSerializer.getSerializedSize(data);
+        } catch (Exception e){
+            callback.addFailed(new ManagedLedgerInterceptException(e), ctx);
+            return;
+        }
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            return;
+        }
+        try {
+            // Why should try-catch here?
+            // If the recycle mechanism is not executed as expected, exception occurs.
             flushContext.addCallback(callback, ctx);
-            bytesSize += dataLength;
-            shouldCompensateCallBackWhenWriteFail = false;
-            trigFlushIfReachMaxRecordsOrMaxSize();
         } catch (Exception e){
-            if (shouldCompensateCallBackWhenWriteFail){
-                callback.addFailed(new ManagedLedgerInterceptException(e), ctx);
-            } else {
-                log.error("Failed to add data asynchronously", e);
-            }
+            callback.addFailed(new ManagedLedgerInterceptException(e), ctx);
+            return;
         }
+        dataArray.add(data);
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();

Review Comment:
   💯 This is *so* much 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] tjiuming commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   @asafm @poorbarcode Oh sorry, I didn't noticed that a metricName is follow the pattern `%s_bufferedwriter_batch_record_count`, then it won't lead to an Exception.
   
   Overall LGTM



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);

Review Comment:
   > I don't understand. What do you mean by refresh?
   
   Misnomer, it should be “flush”.
   
   > Why not do something a bit simpler:
   
   Already fixed
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -541,8 +539,53 @@ public void recycle(){
             this.asyncAddArgsList.clear();
             this.handle.recycle(this);
         }
+
+        public void addCallback(AddDataCallback callback, Object ctx){
+            AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+            asyncAddArgsList.add(asyncAddArgs);
+        }
     }
 
+    /** Callback for batch write BK. **/
+    private final BufferedAddEntryCallback bufferedAddEntryCallback = new BufferedAddEntryCallback();

Review Comment:
   Thank you for your example, already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   
   > I asked if you will remove the if metrics != null statements in the next PRs.
   
   Yes, I will


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   > Why flushContext.asyncAddArgsList.size() and not dataArray.size()?
   
   In callback method `addComplete` after Managed Ledger writes, we can only use `flushContext.asyncAddArgsList.size()` to determine how many records are in the batch, so the same expression is used for all locations



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);

Review Comment:
   > Why Unpooled and not PulsarByteBufAllocator?
   
   Method `Unpooled.wrappedUnmodifiableBuffer` returns just a value object, this is much lighter than `Pooledxxx.compositeBuffer' returns



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.

Review Comment:
   You happen to know the code quite well because you just wrote it a month ago and you forgot. Imagine a developer which is not you - he will most probably forget - this is why comments should be used only when you can't understand the code without them.
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   > But it will break the order, no? If you have 3 records need to write to the managed ledger, A, B, and C. If A fails here, B and C will continue to write. As you said the request hold in the memory queue, the next round A will be retry.
   
   If A already append to the array, it should be ok.
   
   I added a logic that fails callback if the append to the queue fails
   
   > As you said the request hold in the memory queue, the next round A will be retry.
   
   No, If  A is put into the queue successfully, A will be called back earlier than B and C



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,6 +153,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
+        if (batchedWriteMaxRecords <= 1 && batchEnabled){
+            log.warn("The current maximum number of records per batch is set to 1. Disabling Batch will improve"

Review Comment:
   Perhaps "Transaction Log Buffered Writer has batching enabled yet the maximum batch size was configured to less than or equal to 1 record, hence due to performance reasons batching is disabled"
   
   Could it be they wouldn't know per that log line which configuration to change since this can be used in Pending Ack Store and TxLog?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -217,9 +224,9 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                 // Avoid missing callback, do failed callback when error occur before add data to the array.
                 int recordsCountAfter = dataArray.size();
                 if (recordsCountAfter == recordsCountBeforeAdd){
-                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);
+                    callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   Your condition above to only call the callback if number of records in data array haven't changed has a bug.
   Look in internalAsyncAddData:
   ```java
           if (dataLength >= batchedWriteMaxSize){
               trigFlushByLargeSingleData();
               ByteBuf byteBuf = dataSerializer.serialize(data);
               managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                       AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
               return;
           }
   ```
   What happened if asyncAddEntry fails from some reason? You lose the records `data` and you're notifying this to the callback



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -69,6 +70,10 @@
                     new Exception("Transaction log buffered write has closed")
             );
 
+    private static final AtomicReferenceFieldUpdater<TxnLogBufferedWriter, TxnLogBufferedWriter.State> STATE_UPDATER =

Review Comment:
   Something doesn't add up here. In order to prevent race conditions upon updates to `state` STATE_UPDATER was created. Why then some updates are done without it?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,6 +153,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
+        if (batchedWriteMaxRecords <= 1 && batchEnabled){
+            log.warn("The current maximum number of records per batch is set to 1. Disabling Batch will improve"

Review Comment:
   Thinking out loud - @codelipenghui - do you think it's better to do this type of validation and change when processing configuration?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -217,9 +224,9 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                 // Avoid missing callback, do failed callback when error occur before add data to the array.
                 int recordsCountAfter = dataArray.size();
                 if (recordsCountAfter == recordsCountBeforeAdd){
-                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);
+                    callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   I don't understand why you chose the Intercept exception - this is not the error you're witnessing. You tried to asynchronously add data to the buffer and failed. Why not create a new exception if you can't find a similar exception?
   @codelipenghui WDYT?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -217,9 +224,9 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                 // Avoid missing callback, do failed callback when error occur before add data to the array.
                 int recordsCountAfter = dataArray.size();
                 if (recordsCountAfter == recordsCountBeforeAdd){
-                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);
+                    callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   If you're notifying through the callback I'm not sure we actually need a log line in this case. I expect the callback to notify the log, no?
   



-- 
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 #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode closed pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

Posted by GitBox <gi...@apache.org>.
poorbarcode closed pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer
URL: https://github.com/apache/pulsar/pull/16758


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   CI does not work properly, so `rebase Master`


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){
+                log.error("Failed to add data asynchronously, and do failed callback when error occur before add"
+                        + " data to the array.", e);
+                callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   Already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.

Review Comment:
   > This suggestion applies to all comments in this class.
   
   I have checked whether all the comments are reasonable.



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm
   
   I've taken care of all the comments, could you review this PR again?


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
+            managedLedger.asyncAddEntry(wholeByteBuf, this, flushContext);
         }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
+        // Reset the cache.
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();

Review Comment:
   Already create a new class for `callback`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.

Review Comment:
   > Look here, you called pairByteBuf, but not it's wholeByteBuf so the reader wouldn't really understand this comment.
   
   I forgot to change the comment when changing the variable name to meaningful.
   
   > and the delete this comment, which is also duplicated in
   
   Already delete this comment.



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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   I actually don't see any compilation error when accessing `dataArray` from `addComplete` :) ok
   Actually it got me thinking:
   
   ```java
               this.asyncAddArgsList = new ArrayList<>(8);
   ```
   
   ArrayList is not thread-safe. How do you in all of this async code can guarantee no race condition or weird behavior when using this list?
   
   
   
   



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   Can't you just create your own single thread executor service which is ScheduledExecutorService and use it for the scheduling of period flush instead of implementing it on your own using Timer?
   If I understand correctly the whole problem in 16679 was that both the scheduled flush and other triggers for flush were sharing the same thread. So create your own thread but do it using ScheduledExecutorService and save your self from implementing it on your own using timer.
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);

Review Comment:
   Already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);

Review Comment:
   > import static assert
   
   Already fixed.
   
   > Wonder why we don't use AssertJ as in assertThat(addDataCallbackFailureCount.get()).equalsTo(0)
   
   Good idea. I will use it after importing dependencies



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   > I don't see where the `TxnLogBufferedWriterMetricsDefinition.java`'s `labelNames` and `labelValues` has been set, except in the tests.
   
   - `TxnLogBufferedWriterMetricsDefinition` will create by `MlTransactionLogImpl` and `MlPendingAckStore` in next PR.
   - In tests `TxnLogBufferedWriterTest`, `labelNames` and `labelValues` is set by Initializes block which is at top of the class.


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea. Already change the name of the variable to `recordsPerBatchMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some comments here.
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'metricsStats' and these' null checks' will be removed in the next PR. This is compatible only with split PR, making each PR have less code



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some comments here.
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR. This is compatible only with split PR, making each PR have less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some comments here.
   To reduce the cost of maintenance later, I didn't want to support disabled metrics stat in the design, even if the Txn Buffered Writer disabled the batch feature. This constructor without `metricsStats` and the "null checks" will be removed in the next PR. This is only compatible to split the PR, making each PR has less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it only used in the constructor.
   
   Yes, we need.
   
   To build MetricsStat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`. This step can also be omitted, because we can execute `collector.labels(labelValues)` to get it. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we need to get the `Collector` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like `collectorRegistry.getRegistedCollector(String name)`, and it will throws IllegalArgumentException when we registering collector with the same name more than once, see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +620,322 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends JsonDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * 1. Verify Transaction buffered writer stats correct when enabled batch feature. Exclusive "triggerByForceFlush",
+     *    this property verified by {@link #testMetricsStatsWhenForceFlush()}.
+     * 2. Verify metrics will be release after {@link TxnLogBufferedWriterMetricsStats#clone()}.
+     */
+    @Test
+    public void testMetricsStatsWhenEnabled() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger to get the count of refresh event.
+        AtomicInteger refreshCount = new AtomicInteger();
+        String managedLedgerName = "-";
+        ManagedLedger managedLedger = Mockito.mock(ManagedLedger.class);

Review Comment:
   Good idea. already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -60,8 +61,14 @@
 import org.testng.annotations.Test;
 
 @Slf4j
+@Test(groups = "broker")

Review Comment:
   > If you have a scenario for batchRecordCount > max records flush, then check metric there, no?
   
   U are right. Now we have three ways of writing test code: 
   
   - Understand all the internal logic of one method and write tests based on the logic of the code.
   - We are only concerned with inputs and outputs. For example, input 1+1, return 2. We don't care what the internal logic is, we only care about whether the outcome is correct.
   - Combining the first two approaches: we simulate multiple inputs based on known logic and then verify that the output is correct.
   
   I think the third way is the best, there have two ways to do it.
   
    e.g. the logic has two conditions: [a, b]. 
   
   - way-1: we should write 4 unit test method
     - a = true, b = true
     - a = false, b = false
     - a = true, b = false
     - a = false, b = true
   - way-2: we should write only 1 unit test method
     - write a test that handles all scenarios
     - create a data provider: [{true,true}, {false, false}, {true, false}, {false, true}]
     
   If the logic has too many conditions, then the second implementation will have the advantage of being easier to read and maintain.
   
   I will write a separate PR to tidy up the test method `testMainProcess` and make it easier to understand, 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] codelipenghui commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,6 +153,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
+        if (batchedWriteMaxRecords <= 1 && batchEnabled){
+            log.warn("The current maximum number of records per batch is set to 1. Disabling Batch will improve"

Review Comment:
   > Thinking out loud - @codelipenghui - do you think it's better to do this type of validation and change when processing configuration?
   
   Yes, I support having a validation when processing configuration. E.g. batchedWriteMaxRecords >= 10, size >= 128k



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){
+                log.error("Failed to add data asynchronously, and do failed callback when error occur before add"
+                        + " data to the array.", e);
+                callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   I want definition the new class like this
   
   ```java
   public class TxnPersistentException extends Exception {
   
     public TxnPersistentException(String msg, Exception e){
       super(msg, e);
     }
   }
   
   
   ```



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -118,84 +107,71 @@ public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelName
 
         String recordsPerBatchMetricName =
                 String.format("%s_bufferedwriter_batch_record_count", metricsPrefix);
-        recordsPerBatchMetric = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
-                recordsPerBatchMetricName,
-                k -> new Histogram.Builder()
+        this.recordsPerBatchMetric = new Histogram.Builder()

Review Comment:
   No need for `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] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   Hi @asafm 
   
   For E.g. we write these data:
   
   ```java
   // max batch size = 100, max batch records = 5, max delay = 10 milliseconds
   write data_01{ size = 2}
   write data_02{ size = 2}
   write data_03{ size = 2}
   write data_04{ size = 2}
   write data_05{ size = 2}
   write data_06{ size = 111}
   ```
   We have two plans of metrics: include `data_06` or not.
   
   - Exclude `data_06`
     - avg ( batch records count ) = 5
     - avg ( batch size) = 10
   - Include `data_06`
     - avg ( batch records count ) = 3
     - avg ( batch size) = 60
   
   The first plan(exclude data_06) presents metrics that make more sense for adjusting the threshold.
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        verifyTheCounterMetrics(writeCount / 2,0,0,0);

Review Comment:
   > writeCount / maxRecordPerBatch
   
   Correct. I have replaced these expressions with meaningful variable names.



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/
+        verifyTheCounterMetrics(writeCount / 2,0,0,0);

Review Comment:
   > writeCount / maxRecordPerBatch
   
   Correct. I have replaced these expressions with meaningful variable names. Thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);

Review Comment:
   > Here you chose not to notify the callbacks because you assume the queue is still with elements, next timer task will flush them? @codelipenghui WDYT?
   
   It simply prints all the exceptions that have occurred in asynchronous tasks and are used to locate problems.
   
   As long as the request is in the memory queue, it will eventually succeed or fail



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,

Review Comment:
   Already fixed, thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.

Review Comment:
   Already deleted it.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -541,8 +545,52 @@ public void recycle(){
             this.asyncAddArgsList.clear();
             this.handle.recycle(this);
         }
+
+        public void addCallback(AddDataCallback callback, Object ctx){
+            AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+            asyncAddArgsList.add(asyncAddArgs);
+        }
     }
 
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();

Review Comment:
   Good idea. I have moved both `bookKeeperBatchedWriteCallback` and `timingFlushTask` to the variables-place and moved `STATE_UPDATER` to the static-variables-place



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){
+                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);

Review Comment:
   I'm not referring to static variable, but to add an import static statement, so you can only `new ManagedLedgerFencedException(` without the `ManagedLedgerException` prefix.
   
   >No exception is thrown in the block of "try catch". If it does, it will be due to coding errors such as data Serializer or metrics.
   I don't understand. You could have chosen RuntimeException or any other exception class, why did you specifically choose ManagedLedgerFencedException? 
   @codelipenghui this sound reasonable to you?
   



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){

Review Comment:
   You are correct. 
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")

Review Comment:
   Thank you, the description of `help` has been changed



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")

Review Comment:
   Thank you, the description of `help` has been changed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")
+                        .buckets(MAX_DELAY_TIME_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogOldestRecordDelayTimeSecondsChild =
+                pulsarBatchedLogOldestRecordDelayTimeSeconds.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByRecordsName =
+                String.format("%s_batched_log_triggering_count_by_records", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByRecords = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByRecordsName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByRecordsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxRecords\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByRecordsChild = pulsarBatchedLogTriggeringCountByRecords.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountBySizeName =
+                String.format("%s_batched_log_triggering_count_by_size", metricsPrefix);
+        pulsarBatchedLogTriggeringCountBySize = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountBySizeName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountBySizeName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold \"batchedWriteMaxSize\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountBySizeChild = pulsarBatchedLogTriggeringCountBySize.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByDelayTimeName =
+                String.format("%s_batched_log_triggering_count_by_delay_time", metricsPrefix);
+        pulsarBatchedLogTriggeringCountByDelayTime = (Counter) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogTriggeringCountByDelayTimeName,
+                k -> new Counter.Builder()
+                        .name(pulsarBatchedLogTriggeringCountByDelayTimeName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many batches were triggered due to threshold"
+                                + " \"batchedWriteMaxDelayInMillis\"")
+                        .register(registry));
+        pulsarBatchedLogTriggeringCountByDelayTimeChild =
+                pulsarBatchedLogTriggeringCountByDelayTime.labels(labelValues);
+
+        String pulsarBatchedLogTriggeringCountByForcename =

Review Comment:
   Yes, I have changed these variable name



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.
         this.dataArray.add(data);
         // Add callback info.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
         this.flushContext.asyncAddArgsList.add(asyncAddArgs);
         // Calculate bytes-size.
         this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        // trig flush by max records or max size.
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            if (flushContext.asyncAddArgsList.isEmpty()) {
+                return;
+            }
+            if (metrics != null) {
+                metrics.triggerFlushByByMaxDelay(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            // Start the next timing task.
+            nextTimingTrigger();
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
+            if (metrics != null) {
+                metrics.triggerFlushByRecordsCount(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize) {
+            if (metrics != null) {
+                metrics.triggerFlushByBytesSize(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+        }
+    }
+
+    /**
+     * If method {@link #asyncAddData(Object, AddDataCallback, Object)} accept a request that {@param data} is too
+     * large (larger than {@link #batchedWriteMaxSize}), then two flushes:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Directly write the large data to BK, this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private void trigFlushByLargeSingleData(T data, AddDataCallback callback, Object ctx){
+        if (!flushContext.asyncAddArgsList.isEmpty()) {
+            if (metrics != null) {
+                metrics.triggerFlushByLargeSingleData(this.flushContext.asyncAddArgsList.size(), this.bytesSize,
+                        System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+            }
+            doFlush();
+        }
+        ByteBuf byteBuf = dataSerializer.serialize(data);

Review Comment:
   Good suggestion, already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;

Review Comment:
   Good idea.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -89,15 +89,15 @@
     /**
      * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
      */
-    private final int batchedWriteMaxRecords;
+    final int batchedWriteMaxRecords;

Review Comment:
   > What's the motivation for the removal of private from these three variables?
   
   These changes have been restored. Thanks



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -89,15 +89,15 @@
     /**
      * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
      */
-    private final int batchedWriteMaxRecords;
+    final int batchedWriteMaxRecords;

Review Comment:
   These changes have been restored. Thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -144,8 +154,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
         this.flushContext = FlushContext.newInstance();
         this.dataArray = new ArrayList<>();
         this.state = State.OPEN;
-        this.timer = timer;
+        // Metrics.
+        this.metricsStats = metricsStats;
         // scheduler task.
+        this.timer = timer;

Review Comment:
   Already move the comment to its original place, Thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   > What you wrote is there two 2 separate threads using asyncAddArgsList no?
   
   While the second thread holds the reference to `asyncAddArgsList `, the first thread has lost the reference to `asyncAddArgsList `, so there will never be concurrent access



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   `asyncAddArgsList` was modified in two progress: 
   
   - add element: always running in one thread `TxnBufferedWriter.singleThreadExecutorForWrite`
   - read elements and clear: always running in one thread `Bk client`
   
   There is absolutely no crossover between the two processes.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   > Can't you just create your own single thread executor service which is ScheduledExecutorService and use it for the scheduling of period flush instead of implementing it on your own using Timer?
   
   Using 'ScheduledExecutorService' could not make code less, because we can't use the method `scheduleWithFixedDelay`. PR #16679 explains why.
   
   We're using a `Timer` just to reuse objects that already exist.
   
   PR 16679 focuses on the problem: avoiding a large number of "fixed-time flush" tasks in the IO thread
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);
                     return;
                 }
                 // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
                 // fail.
-                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                failureCallbackByContextAndRecycle(flushContext,
+                        new ManagedLedgerException.ManagedLedgerFencedException(
+                            new Exception("Transaction log buffered write has closed")
+                        ));
                 // Cancel the timing task.
-                if (timeout == null) {
-                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                            + " is null. managedLedger: " + managedLedger.getName());
-                } else if (timeout.isCancelled()) {
-                    // TODO How decisions the timer-task has been finished ?
-                    STATE_UPDATER.set(this, State.CLOSED);
-                } else {
-                    if (this.timeout.cancel()) {
-                        STATE_UPDATER.set(this, State.CLOSED);
-                    } else {
-                        // Cancel task failure, The state will stay at CLOSING.
-                        log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
-                                + " stay at CLOSING. managedLedger: " + managedLedger.getName());
-                    }
+                if (!timeout.isCancelled()){
+                    this.timeout.cancel();
                 }
+                STATE_UPDATER.set(this, State.CLOSED);
+                closeFuture.complete(null);
             } catch (Exception e){
                 log.error("Close Txn log buffered writer fail", e);
+                closeFuture.completeExceptionally(e);

Review Comment:
   Already remove this log line



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);

Review Comment:
   You are right, already delete this check



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Txn pending ack batched write max records suggestion at least 10");

Review Comment:
   Already fixed



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   Hi @tjiuming @asafm 
   
   Yes, this has been discussed.
   
   TxnLogBufferedWriterMetricsStats only two instances, they are holds by `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider`.  In the following PR, there will be changes like this: 
   
   - Append modifier `abstract` to `TxnLogBufferedWriterMetricsStats`
   - Create an internal class in `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider` like this:
   
   ```java
   private static class MLTransactionMetadataStoreBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTransactionMetadataStoreBufferedWriterMetrics() {
               super("pulsar_txn_tc", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   
   private static class MLPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTransactionMetadataStoreBufferedWriterMetrics() {
               super("pulsar_txn_pending_ack_store", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   ```
   
   How are you feeling?



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   Hi @tjiuming @asafm 
   
   Yes, this has been discussed.
   
   TxnLogBufferedWriterMetricsStats only two instances, they are holds by `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider`.  In the following PR, there will be changes like this: 
   
   - Append modifier `abstract` to `TxnLogBufferedWriterMetricsStats`
   - Create an internal class in `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider` like this:
   
   ```java
   private static class MLTransactionMetadataStoreBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTransactionMetadataStoreBufferedWriterMetrics() {
               super("pulsar_txn_tc", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   ```
   
   How are you feeling?



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -217,9 +224,9 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                 // Avoid missing callback, do failed callback when error occur before add data to the array.
                 int recordsCountAfter = dataArray.size();
                 if (recordsCountAfter == recordsCountBeforeAdd){
-                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);
+                    callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   Hi @asafm  @codelipenghui 
   
   Since this PR is already too large, I will create a new Exception in the following PR to solve it. Thanks



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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){
+                log.error("Failed to add data asynchronously, and do failed callback when error occur before add"

Review Comment:
   Why are you writing a log line if you activate the callback?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,138 +353,65 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
-        // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteBuf, contentByteBuf);
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
-        }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
-    }
-
-    /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
-     */
-    @Override
-    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        try {
-            final int batchSize = flushContext.asyncAddArgsList.size();
-            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
-                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
-                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
-                        batchIndex);
-                // Because this task already running at ordered task, so just "run".
-                try {
-                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
-                } catch (Exception e){
-                    log.error("After writing to the transaction batched log complete, the callback failed."
-                            + " managedLedger: " + managedLedger.getName(), e);
-                }
-            }
-        } finally {
-            flushContext.recycle();
+            managedLedger.asyncAddEntry(wholeByteBuf, bookKeeperBatchedWriteCallback, flushContext);
         }
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();
+        bytesSize = 0;
     }
 
     /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
-     */
-    @Override
-    public void addFailed(ManagedLedgerException exception, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        failureCallbackByContextAndRecycle(flushContext, exception);
-    }
-
-    /**
-     * Cancel pending tasks and release resources.
+     * Release resources and cancel pending tasks.
      */
     @Override
     public void close() {
-        // If disabled batch feature, there is no closing state.
+        // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
             return;
         }
-        // Prevent the reentrant.
+        // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            // Other thread also calling "close()".
             return;
         }
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
-            if (state == State.CLOSED){
-                return;
-            }
-            // Failure callback to pending request.
-            // If some request has been flushed, Bookie triggers the callback.
-            failureCallbackByContextAndRecycle(this.flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
-            // Cancel task that schedule at fixed rate trig flush.
-            if (timeout == null){
-                log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                        + " is null. managedLedger: " + managedLedger.getName());
-            } else if (timeout.isCancelled()){
-                // TODO How decisions the timer-task has been finished ?
-                this.state = State.CLOSED;
-            } else {
-                if (this.timeout.cancel()) {
-                    this.state = State.CLOSED;
+            try {
+                if (state == State.CLOSED) {
+                    return;
+                }
+                // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
+                // fail.
+                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                // Cancel the timing task.
+                if (timeout == null) {
+                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
+                            + " is null. managedLedger: " + managedLedger.getName());
+                } else if (timeout.isCancelled()) {
+                    // TODO How decisions the timer-task has been finished ?
+                    STATE_UPDATER.set(this, State.CLOSED);
                 } else {
-                    // Cancel task failure, The state will stay at CLOSING.
-                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
-                            + " stay at CLOSING. managedLedger: " + managedLedger.getName());
+                    if (this.timeout.cancel()) {
+                        STATE_UPDATER.set(this, State.CLOSED);
+                    } else {
+                        // Cancel task failure, The state will stay at CLOSING.
+                        log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"

Review Comment:
   The only reason `this.timeout.cancel()` will return false is if the state is not INIT.
   So either state is CANCELLED, which in this case you're good (you can even check that again inside, or the state is EXPIRED which means the task is going to be executed or is being executed now. In this case, since we're running almost everything in the singleThreadExecutorService, one that timer task will get executed, it will see Timeout is canceled and do nothing.
   So just recheck if canceled, if so, close. If not cancelled, close :)
   
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    public void close() {

Review Comment:
   move below constructor please
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   Ok. Tricky one but correct. 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);

Review Comment:
   This is still valid



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,138 +353,65 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
-        // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteBuf, contentByteBuf);
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
-        }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
-    }
-
-    /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
-     */
-    @Override
-    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        try {
-            final int batchSize = flushContext.asyncAddArgsList.size();
-            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
-                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
-                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
-                        batchIndex);
-                // Because this task already running at ordered task, so just "run".
-                try {
-                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
-                } catch (Exception e){
-                    log.error("After writing to the transaction batched log complete, the callback failed."
-                            + " managedLedger: " + managedLedger.getName(), e);
-                }
-            }
-        } finally {
-            flushContext.recycle();
+            managedLedger.asyncAddEntry(wholeByteBuf, bookKeeperBatchedWriteCallback, flushContext);
         }
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();
+        bytesSize = 0;
     }
 
     /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
-     */
-    @Override
-    public void addFailed(ManagedLedgerException exception, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        failureCallbackByContextAndRecycle(flushContext, exception);
-    }
-
-    /**
-     * Cancel pending tasks and release resources.
+     * Release resources and cancel pending tasks.
      */
     @Override
     public void close() {
-        // If disabled batch feature, there is no closing state.
+        // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
             return;
         }
-        // Prevent the reentrant.
+        // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            // Other thread also calling "close()".
             return;
         }
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
-            if (state == State.CLOSED){
-                return;
-            }
-            // Failure callback to pending request.
-            // If some request has been flushed, Bookie triggers the callback.
-            failureCallbackByContextAndRecycle(this.flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
-            // Cancel task that schedule at fixed rate trig flush.
-            if (timeout == null){
-                log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                        + " is null. managedLedger: " + managedLedger.getName());
-            } else if (timeout.isCancelled()){
-                // TODO How decisions the timer-task has been finished ?
-                this.state = State.CLOSED;
-            } else {
-                if (this.timeout.cancel()) {
-                    this.state = State.CLOSED;
+            try {
+                if (state == State.CLOSED) {
+                    return;
+                }
+                // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
+                // fail.
+                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                // Cancel the timing task.
+                if (timeout == null) {
+                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"

Review Comment:
   I actually don't understand the exact error here. How can timeout be null?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){

Review Comment:
   It's always false



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   Summarizing our discussion in Slack: All code using data array is done in a single thread through singleThreadExecutorService to avoid data race. Timeout which reads from the same data structure also needs to be executed via the singleThreadExecutorService. The only way to ensure we don't overlook timing flush check in singleThreadExecutorService is to time the next one *once* the original check is finished, thus Timer is perfectly suited for that.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){
+                log.error("Failed to add data asynchronously, and do failed callback when error occur before add"
+                        + " data to the array.", e);
+                callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   Make sure that in your next PR the exception name would mean exactly what failed since there is not message



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,138 +353,65 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
-        // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteBuf, contentByteBuf);
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
-        }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
-    }
-
-    /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
-     */
-    @Override
-    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        try {
-            final int batchSize = flushContext.asyncAddArgsList.size();
-            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
-                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
-                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
-                        batchIndex);
-                // Because this task already running at ordered task, so just "run".
-                try {
-                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
-                } catch (Exception e){
-                    log.error("After writing to the transaction batched log complete, the callback failed."
-                            + " managedLedger: " + managedLedger.getName(), e);
-                }
-            }
-        } finally {
-            flushContext.recycle();
+            managedLedger.asyncAddEntry(wholeByteBuf, bookKeeperBatchedWriteCallback, flushContext);
         }
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();
+        bytesSize = 0;
     }
 
     /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
-     */
-    @Override
-    public void addFailed(ManagedLedgerException exception, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        failureCallbackByContextAndRecycle(flushContext, exception);
-    }
-
-    /**
-     * Cancel pending tasks and release resources.
+     * Release resources and cancel pending tasks.
      */
     @Override
     public void close() {
-        // If disabled batch feature, there is no closing state.
+        // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
             return;
         }
-        // Prevent the reentrant.
+        // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            // Other thread also calling "close()".
             return;
         }
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
-            if (state == State.CLOSED){
-                return;
-            }
-            // Failure callback to pending request.
-            // If some request has been flushed, Bookie triggers the callback.
-            failureCallbackByContextAndRecycle(this.flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
-            // Cancel task that schedule at fixed rate trig flush.
-            if (timeout == null){
-                log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                        + " is null. managedLedger: " + managedLedger.getName());
-            } else if (timeout.isCancelled()){
-                // TODO How decisions the timer-task has been finished ?
-                this.state = State.CLOSED;
-            } else {
-                if (this.timeout.cancel()) {
-                    this.state = State.CLOSED;
+            try {
+                if (state == State.CLOSED) {
+                    return;
+                }
+                // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
+                // fail.
+                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                // Cancel the timing task.
+                if (timeout == null) {
+                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
+                            + " is null. managedLedger: " + managedLedger.getName());
+                } else if (timeout.isCancelled()) {
+                    // TODO How decisions the timer-task has been finished ?
+                    STATE_UPDATER.set(this, State.CLOSED);
                 } else {
-                    // Cancel task failure, The state will stay at CLOSING.
-                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
-                            + " stay at CLOSING. managedLedger: " + managedLedger.getName());
+                    if (this.timeout.cancel()) {
+                        STATE_UPDATER.set(this, State.CLOSED);
+                    } else {
+                        // Cancel task failure, The state will stay at CLOSING.
+                        log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
+                                + " stay at CLOSING. managedLedger: " + managedLedger.getName());
+                    }
                 }
+            } catch (Exception e){
+                log.error("Close Txn log buffered writer fail", e);

Review Comment:
   Actually, I'm not sure it makes sense here - why would you not want to propagate it upwards?
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){

Review Comment:
   already fixed, thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){
+                log.error("Failed to add data asynchronously, and do failed callback when error occur before add"

Review Comment:
   Already deleted it



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,210 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import lombok.Data;
+import org.apache.commons.collections4.CollectionUtils;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+@Data
+public class TxnLogBufferedWriterMetricsStats {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    @VisibleForTesting
+    static final HashMap<String, TxnLogBufferedWriterMetricsStats> METRICS_REGISTRY = new HashMap<>();
+    /**
+     * Marks all references to instance {@link TxnLogBufferedWriterMetricsStats}, after all objects that depend on the
+     * {@link TxnLogBufferedWriterMetricsStats} are closed, the {@link TxnLogBufferedWriterMetricsStats} will call
+     * {@link CollectorRegistry#unregister(Collector)} for release.
+     */
+    @VisibleForTesting
+    static final HashMap<String, List<TxnLogBufferedWriterMetricsDefinition>> METRICS_INSTANCE_REFERENCE =
+            new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    /** Count of records in per transaction log batch. **/
+    Histogram pulsarBatchedLogRecordsCountPerEntry;
+
+    /** Bytes size per transaction log batch. **/
+    Histogram pulsarBatchedLogEntrySizeBytes;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByRecords;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    Counter pulsarBatchedLogTriggeringCountBySize;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    Counter pulsarBatchedLogTriggeringCountByDelayTime;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,

Review Comment:
   > What do you mean by that explanation? It looks very much the same explanation of pulsarBatchedLogTriggeringCountBySize
   
   Counter:  the data in the queue was flushed prematurely because the next request carried too much data.
   
   I have already renamed this variable to `batchFlushTriggeredByMaxDelayMetric`, and Fixed code comments.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;
+    private final Histogram.Child pulsarBatchedLogOldestRecordDelayTimeSecondsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxRecords}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByRecords;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByRecordsChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxSize}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountBySize;
+    private final Counter.Child pulsarBatchedLogTriggeringCountBySizeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by
+     * {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis}.
+     */
+    private final Counter pulsarBatchedLogTriggeringCountByDelayTime;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByDelayTimeChild;
+
+    /**
+     * The count of the triggering transaction log batch flush actions by force-flush. In addition to manual flush,
+     * force flush is triggered only if the log record bytes size reaches the TxnLogBufferedWriter#batchedWriteMaxSize}
+     * limit.
+     */
+    private final  Counter pulsarBatchedLogTriggeringCountByForce;
+    private final Counter.Child pulsarBatchedLogTriggeringCountByForceChild;
+
+    public void close(){
+        pulsarBatchedLogRecordsCountPerEntry.remove(labelValues);
+        pulsarBatchedLogEntrySizeBytes.remove(labelValues);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByRecords.remove(labelValues);
+        pulsarBatchedLogTriggeringCountBySize.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByDelayTime.remove(labelValues);
+        pulsarBatchedLogTriggeringCountByForce.remove(labelValues);
+    }
+
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,
+                                              CollectorRegistry registry){
+        this.metricsPrefix = metricsPrefix;
+        this.labelNames = labelNames.clone();
+        this.labelValues = labelValues.clone();
+
+        String pulsarBatchedLogRecordsCountPerEntryName =
+                String.format("%s_batched_log_records_count_per_entry", metricsPrefix);
+        pulsarBatchedLogRecordsCountPerEntry = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogRecordsCountPerEntryName,
+                k -> new Histogram.Builder()
+                            .name(pulsarBatchedLogRecordsCountPerEntryName)
+                            .labelNames(labelNames)
+                            .help("A metrics for how many records in per batch")
+                            .buckets(RECORD_COUNT_PER_ENTRY_BUCKETS)
+                            .register(registry));
+
+        pulsarBatchedLogRecordsCountPerEntryChild = pulsarBatchedLogRecordsCountPerEntry.labels(labelValues);
+
+        String pulsarBatchedLogEntrySizeBytesName = String.format("%s_batched_log_entry_size_bytes", metricsPrefix);
+        pulsarBatchedLogEntrySizeBytes = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogEntrySizeBytesName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogEntrySizeBytesName)
+                        .labelNames(labelNames)
+                        .help("A metrics for how many bytes in per batch")
+                        .buckets(BYTES_SIZE_PER_ENTRY_BUCKETS)
+                        .register(registry));
+        pulsarBatchedLogEntrySizeBytesChild = pulsarBatchedLogEntrySizeBytes.labels(labelValues);
+
+        String pulsarBatchedLogOldestRecordDelayTimeSecondsName =
+                String.format("%s_batched_log_oldest_record_delay_time_seconds", metricsPrefix);
+        pulsarBatchedLogOldestRecordDelayTimeSeconds = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
+                pulsarBatchedLogOldestRecordDelayTimeSecondsName,
+                k -> new Histogram.Builder()
+                        .name(pulsarBatchedLogOldestRecordDelayTimeSecondsName)
+                        .labelNames(labelNames)
+                        .help("A metrics for the max latency in per batch")

Review Comment:
   Thank you, the description of `help` has been changed



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   
   > I don't understand this entire explanation. Why does a metric indicating you flushed due to accepting a single large record obscure anything?
   
   These comments are mainly to describe this:
   
   For E.g. we write these data:
   
   ```java
   // max batch size = 100, max batch records = 5, max delay = 10 milliseconds
   write data_01{ size = 2}
   write data_02{ size = 2}
   write data_03{ size = 2}
   write data_04{ size = 2}
   write data_05{ size = 2}
   write data_06{ size = 111}
   ```
   We have two plans of metrics: include `data_06` or not.
   
   - Exclude `data_06`
     - avg ( batch records count ) = 5
     - avg ( batch size) = 10
   - Include `data_06`
     - avg ( batch records count ) = 3
     - avg ( batch size) = 60
   
   The first plan(exclude data_06) presents metrics that make more sense for adjusting the threshold.


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Hi @asafm 
   
   > Can you explain this further? I didn't understand how the provider would hold all the Collectors (metrics). It should only be to a single instance of the metric class.
   
   When `MLTransactionMetadataStoreProvider` initialized, we create `Collector` like this: 
   
   ```java
   public MLTransactionMetadataStoreProvider(){
      this.recordsPerBatchMetric = ...
      this.batchSizeBytesMetric = ...
      this.oldestRecordInBatchDelayTimeSecondsMetric = ...
      this.batchFlushTriggeredByMaxRecordsMetric = ...
      this.batchFlushTriggeredByMaxSizeMetric = ...
      this.batchFlushTriggeredByMaxDelayMetric = ...
   }
   ```
   
   And when creating `MlTransactionLogImpl`, pass these `Collector` to `MlTransactionLogImpl` like this: 
   
   ```java
   public class MLTransactionMetadataStoreProvider{
     public TransactionMetadataStore openStore(...){
       TransactionMetadataStore store = ...;
       setMetrics(store);
       return store;
     }
   
     private void setMetrics(TransactionMetadataStore store) {
       store.recordsPerBatchMetric = this.recordsPerBatchMetric;
       store.batchSizeBytesMetric = this.batchSizeBytesMetric;
       store.oldestRecordInBatchDelayTimeSecondsMetric = this.oldestRecordInBatchDelayTimeSecondsMetric;
       ...
     }
   }
   ```
   
   The `MLTransactionMetadataStoreProvider` will hold all the Collector of Txn Buffered Writer, this is confusing



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -201,30 +213,93 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
         singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
+        // If param-data is too large.
         int len = dataSerializer.getSerializedSize(data);
         if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
-            ByteBuf byteBuf = dataSerializer.serialize(data);
-            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
-                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            trigFlushByLargeSingleData(data, callback, ctx);
             return;
         }
-        // Add data.
+        // Append data to queue.

Review Comment:
   > This suggestion applies to all comments in this class.
   
   Thanks for suggestion, I have checked whether all the comments are reasonable.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = { 10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = { 128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896 };
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = { 1, 5, 10 };
+
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram pulsarBatchedLogRecordsCountPerEntry;
+    private final Histogram.Child pulsarBatchedLogRecordsCountPerEntryChild;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram pulsarBatchedLogEntrySizeBytes;
+    private final Histogram.Child pulsarBatchedLogEntrySizeBytesChild;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final  Histogram pulsarBatchedLogOldestRecordDelayTimeSeconds;

Review Comment:
   Good idea. Already change the name of the variable to `oldestRecordInBatchDelayTimeSecondsMetric`



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -144,8 +154,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
         this.flushContext = FlushContext.newInstance();
         this.dataArray = new ArrayList<>();
         this.state = State.OPEN;
-        this.timer = timer;
+        // Metrics.

Review Comment:
   Good idea. The name of the variable has been changed to `metrics` and removed this code comment.
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.

Review Comment:
   Good idea. Already completed the encapsulation of the constructor.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =

Review Comment:
   Good Idea, already fixed.



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

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

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


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   Why do we need a map here?
   Looks like it only used in the constructor.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some code comments in this PR.
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR( plan 7-2 ). This is compatible only with split PR, making each PR have less code



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some code comments in this PR.
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR( plan 7-2 in Modifications ). This is compatible only with split PR, making each PR have less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it is only used in the constructor.
   
   Yes, we need it.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`.
   
   (<strong> High light</strong>)
    However, the CollectorRegistry does not provide an API like this:
   
   ```java
   public Collector getRegistedCollector(String name);
   ```
   
   and it will throw IllegalArgumentException when we registering collector with the same name more than once, see:
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it only used in the constructor.
   
   Yes, we need.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like this:
   
   ```java
   public Collector getRegistedCollector(String name);
   ```
   
   and it will throw IllegalArgumentException when we registering collector with the same name more than once, see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
+            managedLedger.asyncAddEntry(wholeByteBuf, this, flushContext);
         }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
+        // Reset the cache.

Review Comment:
   Already deleted.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -116,6 +116,16 @@
             AtomicReferenceFieldUpdater
                     .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    /** Metrics. **/
+    private final TxnLogBufferedWriterMetricsStats metricsStats;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor, Timer timer,

Review Comment:
   yes, I will remove the if metrics != null statements in the next PRs.



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -541,8 +539,53 @@ public void recycle(){
             this.asyncAddArgsList.clear();
             this.handle.recycle(this);
         }
+
+        public void addCallback(AddDataCallback callback, Object ctx){
+            AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+            asyncAddArgsList.add(asyncAddArgs);
+        }
     }
 
+    /** Callback for batch write BK. **/
+    private final BufferedAddEntryCallback bufferedAddEntryCallback = new BufferedAddEntryCallback();

Review Comment:
   Let's place this together with all class variables in the beginning of the class
   
   Again, comments: `/** Callback for batch write BK. **/` then
   `bufferedAddEntryCallback` --> `bookKeeperBatchedWriteCallback`
   then delete comment
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.

Review Comment:
   > Ask yourself this please: Can I read the code and understand it without this comment? If yes, delete the comment.
   
   Already delete this comment, 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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);

Review Comment:
   But you are relying on perfect alignment of `Thread.sleep(1)` to `int batchedWriteMaxDelayInMillis = 1;`
   
   1. Connect them: `Thread.sleep(batchedWriteMaxDelayInMillis)`
   2. You can't be sure CI machine running tests won't fluctuate and you will get empty batch hence no flush, right?
   
   Why not do something a bit simpler: 
   1. set maxDelay = 500ms
   2. write 1 record
   3. wait maxDelay*1.2
   4. assert actualFlushCount = 1
   ?
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);
         // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.

Review Comment:
   Yes, bad comments can be misleading. I have understood it and I will gradually establish good habits



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   fix test: `MLPendingAckStoreTest` does not pass the test because we added the check "max record per batch must equal or lager than 10" 


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);
                     return;
                 }
                 // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
                 // fail.
-                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                failureCallbackByContextAndRecycle(flushContext,

Review Comment:
   Sounds 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] tjiuming commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   @asafm Could you please help review the PR?


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:
+ *    [{recordsCount=512, triggerByMaxRecordCount}, {recordCount=1, triggerByTooLarge}], we should not tell the users
+ *    that the average batch records is 256, if the users knows that there are only 256 records per batch, then users
+ *    will try to increase {@link TxnLogBufferedWriter#batchedWriteMaxDelayInMillis} so that there is more data per
+ *    batch to improve throughput, but that does not work.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    /**
+     * Key is the name we used to create {@link TxnLogBufferedWriterMetricsStats}, and now there are two kinds:
+     * ["pulsar_txn_tc_log", "pulsar_txn_tc_batched_log"]. There can be multiple labels in each
+     * {@link TxnLogBufferedWriterMetricsStats}, such as The Transaction Coordinator using coordinatorId as label and
+     * The Transaction Pending Ack Store using subscriptionName as label.
+     */
+    private static final HashMap<String, Collector> COLLECTOR_CACHE = new HashMap<>();

Review Comment:
   > Why do we need a map here? Looks like it only used in the constructor.
   
   Yes, we need.
   
   To build Metrics Stat, we need execute these two steps:
   1. Create `Collector` and register to `CollectorRegistry`, perhaps the Collector is `Histogram` or `Counter` 
   2. Register labels to `Collector` and get `Collector.child`(holds by Metrics Stat). This step can also be omitted, because we can execute `collector.labels(labelValues)` to get `Collector.child`. 
   
   In the Transaction log scenario, multiple Transaction Log share the same `Collector`, and each has its own `Collector.Child`, so when we build metrics stat for each Transaction Log, we call `collector.labels(labelValues)` to get the `Collector.Child`. However, the CollectorRegistry does not provide an API like this(<strong> High light</strong>):
   
   ```java
   public Collector getRegistedCollector(String name);
   ```
   
   and it will throw IllegalArgumentException when we registering collector with the same name more than once(<strong> High light</strong>), see:
   
   
   https://github.com/prometheus/client_java/blob/1966186deaaa83aec496d88ff604a90795bad688/simpleclient/src/main/java/io/prometheus/client/CollectorRegistry.java#L49-L65
   
   So we have to manage the registered collectors ourselves.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -272,23 +290,43 @@ private void doTrigFlush(boolean force, boolean byScheduleThreads){
                 return;
             }
             if (force) {
+                if (metricsStats != null) {

Review Comment:
   > We can add a default TxnLogBufferedWriterMetricsStatsDisabled implementation to avoid many null checks.
   
   Good Idea. In the complete design, we should have two implementations like UML blow, one for enabling the batch feature, and another for disabled:
   
   ![uml](https://user-images.githubusercontent.com/25195800/182418488-6469a38f-a96c-44e9-8ee6-01273b58b0cd.jpeg)
   
   
   Sorry, I should have added some code comments in this PR(It's now written in Modifications).
   To reduce later maintenance costs, I'd like to ditch the 'DisabledMetricsStat' and we'll always use the implementation 'MetricsStatimpl' even if the Txn buffer writer disables batch feature. This constructor without 'param-metricsStats' and these' null checks' will be removed in the next PR( 7-2 in Modifications ). This is compatible only with split PR, making each PR have less code



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   If method {@link #asyncAddData(Object, AddDataCallback, Object)} accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes:
   1. Write the data cached in the queue to BK.
   2. Directly write the large data to BK.
    
   This ensures the sequential nature of multiple writes to BK.
   
   I have rewritten this logic to make it easier to understand.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -60,8 +61,14 @@
 import org.testng.annotations.Test;
 
 @Slf4j
+@Test(groups = "broker")

Review Comment:
   > General comment: The test below don't check each case of trigger. I would expect each scenario to check the metrics to the correct ones. I wouldn't make up a test method to check the metrics which half of them are not really relevant. If you have a scenario for batchRecordCount > max records flush, then check metric there, no?
   
   Thanks, I have split the method `testMetricsStatsWhenDisabledBatchFeature` to this 4 method: 
   
   - `testMetricsStatsThatTriggeredByMaxRecordCount`
   - `testMetricsStatsThatTriggeredByMaxSize`
   - `testMetricsStatsThatTriggeredByMaxDelayTime`
   - `testMetricsStatsThatTriggeredByLargeSingleData`



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm
   
   I've taken care of all the comments, could you review this PR again?


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +608,402 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("txn-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        RandomLenSumStrDataSerializer dataSerializer = new RandomLenSumStrDataSerializer();
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, 256, 1024,
+                1, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * For metrics scenario "max records count".
+     */
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        // Mock managed ledger and write counter.
+        Pair<ManagedLedger, AtomicInteger> mlAndWriteCounter = mockManagedLedgerWithWriteCounter(mlName);
+        ManagedLedger managedLedger = mlAndWriteCounter.getLeft();
+        AtomicInteger batchFlushCounter = mlAndWriteCounter.getRight();
+        // Create callback with counter.
+        Triple<TxnLogBufferedWriter.AddDataCallback, AtomicInteger, AtomicInteger> callbackWithCounter =
+                createCallBackWithCounter();
+        TxnLogBufferedWriter.AddDataCallback addDataCallback = callbackWithCounter.getLeft();
+        AtomicInteger addDataCallbackFinishCount = callbackWithCounter.getMiddle();
+        AtomicInteger addDataCallbackFailureCount = callbackWithCounter.getRight();
+        // Create TxnLogBufferedWriter.
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-threads").build();
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        TxnLogBufferedWriter<Integer> txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                new SumStrDataSerializer(), 2, 1024,
+                1000 * 3600, true, metricsStats);
+        // Add some data.
+        int writeCount = 100;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, addDataCallback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> addDataCallbackFinishCount.get() + addDataCallbackFailureCount.get() == writeCount
+        );
+        Assert.assertEquals(addDataCallbackFailureCount.get(), 0);
+        /** Assert metrics stats correct. **/

Review Comment:
   Already removed this comment. Thanks



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   > Any reason not to replace it with scheduleWithFixedDelay?
   
   see #16679



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        verifyTheCounterMetrics(0,0, actualBatchFlushCount,0);
+        verifyTheHistogramMetrics(actualBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByLargeSingleData() throws Exception {
+        // Use TwoLenSumDataSerializer for: write a little data once, then write a large data once.
+        int bytesSizePerRecordWhichInBatch = 4;
+        int batchedWriteMaxSize = 1024;
+        TwoLenSumDataSerializer dataSerializer =
+                new TwoLenSumDataSerializer(bytesSizePerRecordWhichInBatch, batchedWriteMaxSize);
+        int writeCount = 100;
+        int singleLargeDataRequestCount = writeCount / 2;
+        int expectedBatchFlushTriggeredByLargeData = singleLargeDataRequestCount;
+        int expectedTotalBytesSize = expectedBatchFlushTriggeredByLargeData * bytesSizePerRecordWhichInBatch;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, i);
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(

Review Comment:
   It seems to make no difference. In this unit test, we use `TwoLenSumDataSerializer` to control the size of each write: [4, batchedWriteMaxSize, batchedWriteMaxSize, 4, batchedWriteMaxSize....]
   
   And the code you write is easier to read.



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);

Review Comment:
   > Why do you need this sleep at all?
   
   I want to trigger more flush events, in this unit test. Without sleep,  there will always be only one refresh



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -541,8 +539,53 @@ public void recycle(){
             this.asyncAddArgsList.clear();
             this.handle.recycle(this);
         }
+
+        public void addCallback(AddDataCallback callback, Object ctx){
+            AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+            asyncAddArgsList.add(asyncAddArgs);
+        }
     }
 
+    /** Callback for batch write BK. **/
+    private final BufferedAddEntryCallback bufferedAddEntryCallback = new BufferedAddEntryCallback();

Review Comment:
   Let's place this together with all class variables at the beginning of the class
   
   Again, comments: `/** Callback for batch write BK. **/` then
   `bufferedAddEntryCallback` --> `bookKeeperBatchedWriteCallback`
   then delete comment
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,234 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.Collector;
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import java.util.HashMap;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *    {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature, if a single record is too big, it still directly write to Bookie without batch,
+ *    property {@link #pulsarBatchedLogTriggeringCountByForce} can indicate this case. But this case will not affect
+ *    other metrics, because it would obscure the real situation. E.g. there has two record:

Review Comment:
   Good suggestion, I have already changed the code comment



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -613,9 +610,371 @@ public ByteBuf serialize(ArrayList<Integer> dataArray) {
         }
     }
 
+    private static class RandomLenSumStrDataSerializer extends SumStrDataSerializer {
+
+        @Getter
+        private int totalSize;
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            int size = new Random().nextInt(9);
+            totalSize += size;
+            return size;
+        }
+    }
+
+    private static class TwoLenSumDataSerializer extends JsonDataSerializer {
+
+        private final int len1;
+
+        private final int len2;
+
+        private AtomicBoolean useLen2 = new AtomicBoolean();
+
+        public TwoLenSumDataSerializer(int len1, int len2){
+            this.len1 = len1;
+            this.len2 = len2;
+        }
+
+        /**
+         * After the test, when {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxRecords()} = 256
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxSize()} = 1024,
+         *   and {@link TxnLogBufferedWriterConfig#getBatchedWriteMaxDelayInMillis()} = 1, the random-size baseline
+         *   was set as 9, and there was maximum probability that all three thresholds could be hit.
+         */
+        @Override
+        public int getSerializedSize(Integer data) {
+            boolean b = useLen2.get();
+            useLen2.set(!b);
+            return b ? len2 : len1;
+        }
+    }
+
     public enum BookieErrorType{
         NO_ERROR,
         ALWAYS_ERROR,
         SOMETIMES_ERROR;
     }
+
+    /**
+     * Test Transaction buffered writer stats when disabled batch feature.
+     */
+    @Test
+    public void testMetricsStatsWhenDisabledBatchFeature() throws Exception {
+        TxnLogBufferedWriterMetricsStats metricsStats = new TxnLogBufferedWriterMetricsStats(
+                metricsPrefix, metricsLabelNames, metricsLabelValues, CollectorRegistry.defaultRegistry
+        );
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder().numThreads(5).name("txn-threads").build();
+        // Create TxnLogBufferedWriter.
+        HashedWheelTimer transactionTimer = new HashedWheelTimer(new DefaultThreadFactory("transaction-timer"),
+                1, TimeUnit.MILLISECONDS);
+        var dataSerializer = new RandomLenSumStrDataSerializer();
+        var txnLogBufferedWriter = new TxnLogBufferedWriter<Integer>(
+                managedLedger, orderedExecutor, transactionTimer,
+                dataSerializer, Integer.MAX_VALUE, Integer.MAX_VALUE,
+                Integer.MAX_VALUE, false, metricsStats);
+        // Add some data.
+        int writeCount = 1000;
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        // Assert metrics stat.
+        verifyTheHistogramMetrics(0, 0, 0);
+        // cleanup.
+        txnLogBufferedWriter.close();
+        metricsStats.close();
+        transactionTimer.stop();
+        orderedExecutor.shutdown();
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxRecordCount() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxRecords = 2;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / batchedWriteMaxRecords;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        // Create callback with counter.
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, batchedWriteMaxRecords, Integer.MAX_VALUE, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(expectedBatchFlushCount,0,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxSize() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int batchedWriteMaxSize = 16;
+        int writeCount = 100;
+        int expectedBatchFlushCount = writeCount / (batchedWriteMaxSize / dataSerializer.getSizePerData());
+        int expectedTotalBytesSize = expectedBatchFlushCount * batchedWriteMaxSize;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(1, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        assertEquals(expectedBatchFlushCount, actualBatchFlushCount);
+        verifyTheCounterMetrics(0, expectedBatchFlushCount,0,0);
+        verifyTheHistogramMetrics(expectedBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByMaxDelayTime() throws Exception {
+        SumStrDataSerializer dataSerializer = new SumStrDataSerializer();
+        int writeCount = 100;
+        int expectedTotalBytesSize = writeCount * dataSerializer.getSizePerData();
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext =
+                createTxnBufferedWriterContextWithMetrics(dataSerializer, Integer.MAX_VALUE,
+                        Integer.MAX_VALUE, 1);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, "");
+            Thread.sleep(1);
+        }
+        // Wait for all write finish.
+        Awaitility.await().atMost(5, TimeUnit.SECONDS).until(
+                () -> callbackWithCounter.finishCounter.get() + callbackWithCounter.failureCounter.get() == writeCount
+        );
+        int actualBatchFlushCount = txnLogBufferedWriterContext.mockedManagedLedger.writeCounter.get();
+        assertEquals(callbackWithCounter.failureCounter.get(), 0);
+        verifyTheCounterMetrics(0,0, actualBatchFlushCount,0);
+        verifyTheHistogramMetrics(actualBatchFlushCount, writeCount, expectedTotalBytesSize);
+        // cleanup.
+        releaseTxnLogBufferedWriterContext(txnLogBufferedWriterContext);
+        // after close, verify the metrics change to 0.
+        verifyTheCounterMetrics(0,0,0,0);
+        verifyTheHistogramMetrics(0,0,0);
+    }
+
+    @Test
+    public void testMetricsStatsThatTriggeredByLargeSingleData() throws Exception {
+        // Use TwoLenSumDataSerializer for: write a little data once, then write a large data once.
+        int bytesSizePerRecordWhichInBatch = 4;
+        int batchedWriteMaxSize = 1024;
+        TwoLenSumDataSerializer dataSerializer =
+                new TwoLenSumDataSerializer(bytesSizePerRecordWhichInBatch, batchedWriteMaxSize);
+        int writeCount = 100;
+        int singleLargeDataRequestCount = writeCount / 2;
+        int expectedBatchFlushTriggeredByLargeData = singleLargeDataRequestCount;
+        int expectedTotalBytesSize = expectedBatchFlushTriggeredByLargeData * bytesSizePerRecordWhichInBatch;
+        var callbackWithCounter = createCallBackWithCounter();
+        // Create TxnLogBufferedWriter.
+        var txnLogBufferedWriterContext = createTxnBufferedWriterContextWithMetrics(
+                dataSerializer, Integer.MAX_VALUE, batchedWriteMaxSize, Integer.MAX_VALUE);
+        var txnLogBufferedWriter = txnLogBufferedWriterContext.txnLogBufferedWriter;
+        // Add some data.
+        for (int i = 0; i < writeCount; i++){
+            txnLogBufferedWriter.asyncAddData(1, callbackWithCounter.callback, i);
+        }
+        // Wait for all data write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(

Review Comment:
   > Where are you checking that you wrote to managed ledger writeCount times?
   
   Already add this check



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,64 +336,24 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteFuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteFuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteFuf, contentByteBuf);

Review Comment:
   > So you do get a composite bytebuf, because you want to avoid the copy of course.
   
   'FixedCompositeByteBuf' and 'CompositeByteBuf' are similar names, but they have no relation. `FixedCompositeByteBuf` is a just value object, there is only the overhead of creating the 'FixedCompositeByteBuf$Component' object and the GC overhead, is appropriate for this scenario.
   
   > When you're not using the pulsar version you're missing out on several configurations that might come in handy.
   
   I could not understand 



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   > But it will break the order, no? If you have 3 records need to write to the managed ledger, A, B, and C. If A fails here, B and C will continue to write. As you said the request hold in the memory queue, the next round A will be retry.
   
   If A already append to the array, it should be ok.
   
   I added a logic that fails callback if the append to the queue fails



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){

Review Comment:
   Already fixed



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Txn pending ack batched write max records suggestion at least 10");

Review Comment:
   Already fixed



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

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

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


[GitHub] [pulsar] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Txn pending ack batched write max records suggestion at least 10");

Review Comment:
   1. When the user gets this error, they need to know exactly which property failed them. I think it's a good idea to say explicitly the property name in the error message.
   2. When you use the word "suggestion" you make it sound it sound it's not required to have this value above 10. 
   
   How about:
   "Configuration field '{}' value must be greater than 10 (value configured was '{}')" 
   
   Use the same idea of phrasing for the rest of the error messages



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){

Review Comment:
   1. The name is too generic. Other developers might say "Oh, this is where we make sure our config is appropriate, let's add our validation code in this method" - this method will quickly spin out of control and become one big messy hairball.
   2. The location of this logic inside a class which is key to Pulsar and already too big.
   3. It doesn't follow the previous validation code examples before, in this constructor.
   
   I suggest the following:
   * Move this method into a new class called `TransactionBatchedWriteValidator` (or perhaps a better name) in the package `org.apache.pulsar.broker.validator`
   * rename method to `validate` (since class name already encapsulate the full meaning)
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);

Review Comment:
   What is the scenario in which you can reach state = CLOSED? Only one thread will be able to pass OPEN -> CLOSING and submit the close lambda on the single thread executor. This is the only lambda that can convert CLOSING -> CLOSED. 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);
                     return;
                 }
                 // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
                 // fail.
-                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                failureCallbackByContextAndRecycle(flushContext,

Review Comment:
   This should be `new TxLogBufferedWriterException("Transaction log buffered write is closed")`. Fenced Exception and ManagedLedger are completely unrelated.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -374,46 +373,42 @@ private void doFlush(){
     /**
      * Release resources and cancel pending tasks.
      */
-    @Override
-    public void close() {
+    public CompletableFuture<Void> close() {
         // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
-            return;
+            return CompletableFuture.completedFuture(null);
         }
         // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            return;
+            return CompletableFuture.completedFuture(null);
         }
+        CompletableFuture closeFuture = new CompletableFuture();
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
             try {
                 if (state == State.CLOSED) {
+                    closeFuture.complete(null);
                     return;
                 }
                 // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
                 // fail.
-                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                failureCallbackByContextAndRecycle(flushContext,
+                        new ManagedLedgerException.ManagedLedgerFencedException(
+                            new Exception("Transaction log buffered write has closed")
+                        ));
                 // Cancel the timing task.
-                if (timeout == null) {
-                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                            + " is null. managedLedger: " + managedLedger.getName());
-                } else if (timeout.isCancelled()) {
-                    // TODO How decisions the timer-task has been finished ?
-                    STATE_UPDATER.set(this, State.CLOSED);
-                } else {
-                    if (this.timeout.cancel()) {
-                        STATE_UPDATER.set(this, State.CLOSED);
-                    } else {
-                        // Cancel task failure, The state will stay at CLOSING.
-                        log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
-                                + " stay at CLOSING. managedLedger: " + managedLedger.getName());
-                    }
+                if (!timeout.isCancelled()){
+                    this.timeout.cancel();
                 }
+                STATE_UPDATER.set(this, State.CLOSED);
+                closeFuture.complete(null);
             } catch (Exception e){
                 log.error("Close Txn log buffered writer fail", e);
+                closeFuture.completeExceptionally(e);

Review Comment:
    You can remove the log line above it since you're informing about the exception in the CompletableFuture.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java:
##########
@@ -368,6 +369,31 @@ public MetadataStore createConfigurationMetadataStore(PulsarMetadataEventSynchro
                         .build());
     }
 
+    private static void ensureConfigIsAppropriate(ServiceConfiguration configuration){
+        if (configuration.isTransactionPendingAckBatchedWriteEnabled()){
+            if (configuration.getTransactionPendingAckBatchedWriteMaxRecords() < 10){
+                throw new IllegalArgumentException("Txn pending ack batched write max records suggestion at least 10");
+            }
+            if (configuration.getTransactionPendingAckBatchedWriteMaxSize() < 1024 * 128){

Review Comment:
   @codelipenghui I don't have enough context to judge - are the values validated here make sense?
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +223,102 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            internalAsyncAddData(data, callback, ctx);
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
-        if (state == State.CLOSING || state == State.CLOSED){
-            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+        // Avoid missing callback, do failed callback when error occur before add data to the array.
+        boolean shouldCompensateCallBackWhenWriteFail = false;
+        try {
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            int dataLength = dataSerializer.getSerializedSize(data);
+            if (dataLength >= batchedWriteMaxSize){
+                trigFlushByLargeSingleData();
+                ByteBuf byteBuf = dataSerializer.serialize(data);
+                shouldCompensateCallBackWhenWriteFail = false;
+                managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                        AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+                return;
+            }
+            dataArray.add(data);
+            flushContext.addCallback(callback, ctx);
+            bytesSize += dataLength;
+            shouldCompensateCallBackWhenWriteFail = false;
+            trigFlushIfReachMaxRecordsOrMaxSize();
+        } catch (Exception e){
+            if (shouldCompensateCallBackWhenWriteFail){
+                log.error("Failed to add data asynchronously, and do failed callback when error occur before add"
+                        + " data to the array.", e);
+                callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   I'm looking at this now and I'm thinking the following: The code that called BufferedWriter, asked you to add a record to the buffer, right? It also provided you with a callback so you can notify it whether it has failed or succeeded. Inside this code:
   
   ```java
   } catch (Exception e){
               if (shouldCompensateCallBackWhenWriteFail){
                   callback.addFailed(new ManagedLedgerInterceptException(e), ctx);
               } else {
                   log.error("Failed to add data asynchronously", e);
               }
           }
   ```
   
   I read and re-read the code and the way it's written is super hard to think exactly what will happen, at every scenario. maybe one of the flush methods, maybe it's the doFlush - you have no idea in what state it failed, and whether you should call the callback or not.
   
   My suggestion is to avoid that boolean at all.
   Perhaps it's best to wrap single sections with try catch and act accordingly.
   At the end you have:
   - add record
   - flush if needed
   
   If add record failed, you call the callback.
   the flush needs to be self contained with try catch, and inside you know exactly when something fail and you can call the callbacks of each record.
   I would only leave the outer try catch in the internal-add to log error since it simply should never happen, but if it does (like OOME) you know where it failed.
   



-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   @tjiuming This is what I consulted with you if you remember a couple of weeks ago, and based on this I wrote: https://github.com/apache/pulsar/pull/16758#issuecomment-1207782630



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    /**
+     * Users needs to ensure that the {@link TxnLogBufferedWriterMetricsStats} of the same {@param metricsPrefix} can
+     * only create once, otherwise an IllegalArgumentException will be thrown.
+     */
+    public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelNames, String[] labelValues,

Review Comment:
   Hi @tjiuming @asafm 
   
   Yes, this has been discussed.
   
   TxnLogBufferedWriterMetricsStats only two instances, they are holds by `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider`.  In the following PR, there will be changes like this: 
   
   - Append modifier `abstract` to `TxnLogBufferedWriterMetricsStats`
   - Create an internal class in `MLTransactionMetadataStoreProvider` and `TransactionPendingAckStoreProvider` like this:
   
   ```java
   private static class MLTransactionMetadataStoreBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTransactionMetadataStoreBufferedWriterMetrics() {
               super("pulsar_txn_tc", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   ```
   
   ```java
   private static class MLPendingAckLogBufferedWriterMetrics extends TxnLogBufferedWriterMetricsStats{
   
           private MLTransactionMetadataStoreBufferedWriterMetrics() {
               super("pulsar_txn_pending_ack_store", new String[]{"cluster", "broker"}, new String[]{"xxx", "xxx"}, CollectorRegistry.defaultRegistry);
           }
   }
   ```
   
   How are you feeling?



-- 
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] asafm commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   I started writing a few comments then I realized I have some profound suggestions I thought maybe it's a better idea to raise them first. 
   
   I think currently this PR in my opinion quite complicated. It took me 2 hours to get the hang of it all.
   Here is what I suggest IMO to make it a bit simpler:
   
   1. There is a lot of logic related to removing labels once the buffered writer is closing. It's even a bit more complicated as some buffered writers share the same label values, so they have one child of the histogram collector for example, but many usages. It's very complicated. 
   
   BufferedWriter is used on two occasions: Pending Ack stores, and Transaction Log. 
   Transaction Log doesn't need that complicated logic as it creates a single Buffered Writer.
   Pending Ack Stores does. Since it has a provider class, we can make this logic "applicative" and place it in the `MLPendingAckStoreProvider`. 
   
   How?
   BufferedWriter will get a BufferedWriterMetrics in the constructor.
   BufferedWriterMetrics will get BufferedWriterMetricsConfiguration in its constructor.
   
   BufferedWriterMetricsConfiguration is the same as the definition you have:
   `labelNames`, `labelValues`, `metricsPrefix` (you called it `component`) and `enabledMetrics`.
   
   
   Since you said the pending ack stores will share the same labels, but won't be differentiated by the label values, you can create a single BufferedWriterMetrics instance and use it whenever you create a new Buffered writer.
   
   When a ledger is closed, its buffered writer is closed.
   BufferedWriterMetrics will be closed by its creator: `MLPendingAckStoreProvider` will know when to close it's the only instance since it can easily keep track or already have the number of open ack stores. `TransactionLog` will keep its only instance and will close it upon closing the managed ledger / buffered writer.
   
   2. I wouldn't bother with unregistering the metric. It's only relevant when there are no pending ack stores. The only cost in this case: is 2 lines emitted in the Prometheus output (help and type) since there are no samples to print.
   
   3. In BufferedWriter Metrics init, I would use the same optimization trick I saw at `FunctionStatsManager`: 
    step 1: create all metrics: histogram, collectors, etc. mainly supplying labels names.
   step 2: create the child of each collector, by supplying the label values. Save it as a variable and use it.
   
   4. In close(), just remove the labels from the collector.
   5. One larger change I would do: have BufferedWriterMetrics class have an action method: batchFlushedTriggedByForceFlush(), and its arguments containing anything you need for your metrics update. hide everything inside, including the `appendistogram` you have there. 
   6. I would get rid of the disabled static instance, and simply do nothing upon each action method in BufferedWriterMetrics if `metricsEnabled` is false. Encapsulate it.
   
   I have many more comments, but I thought it's best to discuss the big ones first.
   
   
   
   
   
   


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm
   
   I've taken care of all the comments, could you review this PR again?


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   Hi @asafm 
   
   > ComponentStatsManager
   
   Yes, I know it exists, but there has a package dependency problem, need to pull it into `pulsar-common`.


-- 
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] asafm commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -112,10 +117,31 @@
 
     /** The main purpose of state maintenance is to prevent written after close. **/
     private volatile State state;
-    private static final AtomicReferenceFieldUpdater<TxnLogBufferedWriter, TxnLogBufferedWriter.State> STATE_UPDATER =
-            AtomicReferenceFieldUpdater
-                    .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();
+
+    private final TxnLogBufferedWriterMetricsStats metrics;
+
+    /**
+     * In the {@link #asyncAddData}, exceptions may occur. To avoid losing the callback, use a variable to mark whether
+     * a callback needs to be compensated.
+     */
+    private boolean shouldCallBackWhenWriteFail;

Review Comment:
   That's a very big no-no unless I missed something.
   
   1. That's like the anti-pattern of defining a global variable.
   2. Look at the distances - you're declaring the variable here, updating it in the public add data method, and also updating it inside the internal add data method. That's too messy and hard to keep up.
   3. Your red flag should be raised pretty high when you need 2 full sentences which I honestly don't understand as a comment for a single variable. 
   
   I suggest: 
   1. Delete variable from here
   2. Use the option to return a value for the internal add async method to return that value
   
   



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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -259,138 +353,65 @@ private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
 
     }
 
-    /**
-     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
-     */
-    public void trigFlush(final boolean force, boolean byScheduleThreads){
-        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force, byScheduleThreads));
-    }
-
-    private void doTrigFlush(boolean force, boolean byScheduleThreads){
-        try {
-            if (flushContext.asyncAddArgsList.isEmpty()) {
-                return;
-            }
-            if (force) {
-                doFlush();
-                return;
-            }
-            if (byScheduleThreads) {
-                doFlush();
-                return;
-            }
-            AsyncAddArgs firstAsyncAddArgs = flushContext.asyncAddArgsList.get(0);
-            if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime >= batchedWriteMaxDelayInMillis) {
-                doFlush();
-                return;
-            }
-            if (this.flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {
-                doFlush();
-                return;
-            }
-            if (this.bytesSize >= batchedWriteMaxSize) {
-                doFlush();
-            }
-        } finally {
-            if (byScheduleThreads) {
-                nextTimingTrigger();
-            }
-        }
-    }
-
     private void doFlush(){
-        // Combine data.
-        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
-        prefix.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
-        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
-        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
-        // We need to release this pairByteBuf after Managed ledger async add callback. Just holds by FlushContext.
-        this.flushContext.byteBuf = pairByteBuf;
-        // Flush.
+        // Combine data cached by flushContext, and write to BK.
+        ByteBuf prefixByteBuf = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefixByteBuf.writeShort(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf contentByteBuf = dataSerializer.serialize(dataArray);
+        ByteBuf wholeByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefixByteBuf, contentByteBuf);
+        flushContext.byteBuf = wholeByteBuf;
         if (State.CLOSING == state || State.CLOSED == state){
             failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
         } else {
-            managedLedger.asyncAddEntry(pairByteBuf, this, this.flushContext);
-        }
-        // Clear buffers.ok
-        this.dataArray.clear();
-        this.flushContext = FlushContext.newInstance();
-        this.bytesSize = 0;
-    }
-
-    /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
-     */
-    @Override
-    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        try {
-            final int batchSize = flushContext.asyncAddArgsList.size();
-            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
-                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
-                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
-                        batchIndex);
-                // Because this task already running at ordered task, so just "run".
-                try {
-                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
-                } catch (Exception e){
-                    log.error("After writing to the transaction batched log complete, the callback failed."
-                            + " managedLedger: " + managedLedger.getName(), e);
-                }
-            }
-        } finally {
-            flushContext.recycle();
+            managedLedger.asyncAddEntry(wholeByteBuf, bookKeeperBatchedWriteCallback, flushContext);
         }
+        dataArray.clear();
+        flushContext = FlushContext.newInstance();
+        bytesSize = 0;
     }
 
     /**
-     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
-     */
-    @Override
-    public void addFailed(ManagedLedgerException exception, Object ctx) {
-        final FlushContext flushContext = (FlushContext) ctx;
-        failureCallbackByContextAndRecycle(flushContext, exception);
-    }
-
-    /**
-     * Cancel pending tasks and release resources.
+     * Release resources and cancel pending tasks.
      */
     @Override
     public void close() {
-        // If disabled batch feature, there is no closing state.
+        // If batch feature is disabled, there is nothing to close, so set the stat only.
         if (!batchEnabled) {
             STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
             return;
         }
-        // Prevent the reentrant.
+        // If other thread already called "close()", so do nothing.
         if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
-            // Other thread also calling "close()".
             return;
         }
         // Cancel pending tasks and release resources.
         singleThreadExecutorForWrite.execute(() -> {
-            if (state == State.CLOSED){
-                return;
-            }
-            // Failure callback to pending request.
-            // If some request has been flushed, Bookie triggers the callback.
-            failureCallbackByContextAndRecycle(this.flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
-            // Cancel task that schedule at fixed rate trig flush.
-            if (timeout == null){
-                log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
-                        + " is null. managedLedger: " + managedLedger.getName());
-            } else if (timeout.isCancelled()){
-                // TODO How decisions the timer-task has been finished ?
-                this.state = State.CLOSED;
-            } else {
-                if (this.timeout.cancel()) {
-                    this.state = State.CLOSED;
+            try {
+                if (state == State.CLOSED) {
+                    return;
+                }
+                // If some requests are flushed, BK will trigger these callbacks, and the remaining requests in should
+                // fail.
+                failureCallbackByContextAndRecycle(flushContext, BUFFERED_WRITER_CLOSED_EXCEPTION);
+                // Cancel the timing task.
+                if (timeout == null) {
+                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The field-timeout"
+                            + " is null. managedLedger: " + managedLedger.getName());
+                } else if (timeout.isCancelled()) {
+                    // TODO How decisions the timer-task has been finished ?
+                    STATE_UPDATER.set(this, State.CLOSED);
                 } else {
-                    // Cancel task failure, The state will stay at CLOSING.
-                    log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
-                            + " stay at CLOSING. managedLedger: " + managedLedger.getName());
+                    if (this.timeout.cancel()) {
+                        STATE_UPDATER.set(this, State.CLOSED);
+                    } else {
+                        // Cancel task failure, The state will stay at CLOSING.
+                        log.error("Cancel timeout-task that schedule at fixed rate trig flush failure. The state will"
+                                + " stay at CLOSING. managedLedger: " + managedLedger.getName());
+                    }
                 }
+            } catch (Exception e){
+                log.error("Close Txn log buffered writer fail", e);

Review Comment:
   Good idea, already fixed



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -0,0 +1,208 @@
+/**
+ * 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.transaction.coordinator.impl;
+
+import io.prometheus.client.CollectorRegistry;
+import io.prometheus.client.Counter;
+import io.prometheus.client.Histogram;
+import java.io.Closeable;
+import lombok.Getter;
+
+/***
+ * Describes the working status of the {@link TxnLogBufferedWriter}, helps users tune the thresholds of
+ * {@link TxnLogBufferedWriter} for best performance.
+ * Note-1: When batch feature is turned off, no data is logged at this. In this scenario,users can see the
+ *   {@link org.apache.bookkeeper.mledger.ManagedLedgerMXBean}.
+ * Note-2: Even if enable batch feature. A batch has numerous triggers. The metrics in this class count each type of
+ *   trigger to allow you to diagnose what mostly causing a batch flush. The metric also includes a histogram for delay
+ *   of batch since 1st record entered, the size of batch in bytes number of records in batch. This will help you to
+ *   tune the parameters that control some of the batch flush triggers: maxDelay, maxRecords, maxSize.
+ *   Note that the 4th trigger - a single record larger than batch size - triggers a flush of the current batch, but
+ *   the big record itself is not written in batch hence is not included in the batch metrics written above (batch
+ *   size, batch delay, etc). The trigger is of course counted as other trigger types.
+ */
+public class TxnLogBufferedWriterMetricsStats implements Closeable {
+
+    static final double[] RECORD_COUNT_PER_ENTRY_BUCKETS = {10, 50, 100, 200, 500, 1_000};
+
+    static final double[] BYTES_SIZE_PER_ENTRY_BUCKETS = {128, 512, 1_024, 2_048, 4_096, 16_384,
+            102_400, 1_232_896};
+
+    static final double[] MAX_DELAY_TIME_BUCKETS = {1, 5, 10};
+
+    @Getter
+    private final String metricsPrefix;
+
+    private final String[] labelNames;
+
+    private final String[] labelValues;
+
+    /** Count of records in per transaction log batch. **/
+    private final Histogram recordsPerBatchMetric;
+    private final Histogram.Child recordsPerBatchHistogram;
+
+    /** Bytes size per transaction log batch. **/
+    private final Histogram batchSizeBytesMetric;
+    private final Histogram.Child batchSizeBytesHistogram;
+
+    /** The time of the oldest transaction log spent in the buffer before being sent. **/
+    private final Histogram oldestRecordInBatchDelayTimeSecondsMetric;
+    private final Histogram.Child oldestRecordInBatchDelayTimeSecondsHistogram;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxRecords". **/
+    private final Counter batchFlushTriggeredByMaxRecordsMetric;
+    private final Counter.Child batchFlushTriggeredByMaxRecordsCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxSize". **/
+    private final Counter batchFlushTriggeredByMaxSizeMetric;
+    private final Counter.Child batchFlushTriggeredByMaxSizeCounter;
+
+    /** The count of the triggering transaction log batch flush actions by "batchedWriteMaxDelayInMillis". **/
+    private final Counter batchFlushTriggeredByMaxDelayMetric;
+    private final Counter.Child batchFlushTriggeredByMaxDelayCounter;
+
+    /**
+     * If {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)} accept a
+     * request that param-data is too large (larger than "batchedWriteMaxSize"), then two flushes are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
+    private final Counter batchFlushTriggeredByLargeSingleDataMetric;
+    private final Counter.Child batchFlushTriggeredByLargeSingleDataCounter;
+
+    public void close() {

Review Comment:
   Already fixed.



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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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

   /pulsarbot rerun-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16758: [improve][txn] PIP-160 Metrics stats of Transaction buffered writer

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -210,9 +210,15 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
             return;
         }
         singleThreadExecutorForWrite.execute(() -> {
+            int recordsCountBeforeAdd = dataArray.size();
             try {
                 internalAsyncAddData(data, callback, ctx);
             } catch (Exception e){
+                // Avoid missing callback, do failed callback when error occur before add data to the array.
+                int recordsCountAfter = dataArray.size();
+                if (recordsCountAfter == recordsCountBeforeAdd){
+                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);

Review Comment:
   Hi @asafm 
   
   > Can you please explain why did you choose the Fenced exception for this specific error which you don't know its nature?
   
   I'm sorry I misunderstood. You are correct, should not choose to use the Fenced exception here. 
   
   > I'm not referring to static variable, but to add an import static statement
   
   already fixed, thanks.
   
   Hi @codelipenghui 
   
   > we should not throw ManagedLedgerException out of ManagedLedger.
   IMO, we should use PersistenceException instead.
   
   Could I use `ManagedLedgerException.ManagedLedgerInterceptException` here? because: 
   - `MlTransactionLogImpl` and `MlPendingAckStore` already used `ManagedLedgerException` because they originally use the `void asyncAddEntry(byte[] data, AddEntryCallback callback, Object ctx)` method
   - TxnBufferedWriter cannot access the `PersistenceException` because module `pulsar-transaction` can not import the dependency `broker`
   
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);

Review Comment:
   Already changed it to `Failed to add data asynchronously`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -217,9 +224,9 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                 // Avoid missing callback, do failed callback when error occur before add data to the array.
                 int recordsCountAfter = dataArray.size();
                 if (recordsCountAfter == recordsCountBeforeAdd){
-                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);
+                    callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   Good catch. I have used a variable `shouldCompensateCallBackWhenWriteFail` to mark whether a callback needs to be compensated



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();

Review Comment:
   I mistook you said `java.util.concurrent.ScheduledExecutorService`, I have to read more of `org.apache.bookkeeper.common.util.OrderedScheduler`.
   
   But now using the `timer` can save one thread.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -198,33 +209,99 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                internalAsyncAddData(data, callback, ctx);
+            } catch (Exception e){
+                log.error("Internal async add data fail", e);
+            }
+        });
     }
 
+    /**
+     * Append data to queue, if reach {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush. And if
+     * accept a request that {@param data} is too large (larger than {@link #batchedWriteMaxSize}), then two flushes
+     * are executed:
+     *    1. Write the data cached in the queue to BK.
+     *    2. Direct write the large data to BK,  this flush event will not record to Metrics.
+     * This ensures the sequential nature of multiple writes to BK.
+     */
     private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
         if (state == State.CLOSING || state == State.CLOSED){
             callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
             return;
         }
-        int len = dataSerializer.getSerializedSize(data);
-        if (len >= batchedWriteMaxSize){
-            if (!flushContext.asyncAddArgsList.isEmpty()) {
-                doTrigFlush(true, false);
-            }
+        int dataLength = dataSerializer.getSerializedSize(data);
+        if (dataLength >= batchedWriteMaxSize){
+            trigFlushByLargeSingleData();
             ByteBuf byteBuf = dataSerializer.serialize(data);
             managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
                     AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
             return;
         }
-        // Add data.
-        this.dataArray.add(data);
-        // Add callback info.
+        // Append data to the data-array.
+        dataArray.add(data);
+        // Append callback to the flushContext.
         AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
-        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
-        // Calculate bytes-size.
-        this.bytesSize += len;
-        // trig flush.
-        doTrigFlush(false, false);
+        flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes size.
+        bytesSize += dataLength;
+        trigFlushIfReachMaxRecordsOrMaxSize();
+    }
+
+    /**
+     * Change to IO thread and do flush, only called by {@link #timingFlushTask}.
+     */
+    private void trigFlushByTimingTask(){
+        singleThreadExecutorForWrite.execute(() -> {
+            try {
+                if (flushContext.asyncAddArgsList.isEmpty()) {
+                    return;
+                }
+                if (metrics != null) {
+                    metrics.triggerFlushByByMaxDelay(flushContext.asyncAddArgsList.size(), bytesSize,
+                            System.currentTimeMillis() - flushContext.asyncAddArgsList.get(0).addedTime);
+                }
+                doFlush();
+            } catch (Exception e){
+                log.error("Trig flush by timing task fail.", e);
+            } finally {
+                // Start the next timing task.
+                nextTimingTrigger();
+            }
+        });
+    }
+
+    /**
+     * If reach the thresholds {@link #batchedWriteMaxRecords} or {@link #batchedWriteMaxSize}, do flush.
+     */
+    private void trigFlushIfReachMaxRecordsOrMaxSize(){
+        if (flushContext.asyncAddArgsList.size() >= batchedWriteMaxRecords) {

Review Comment:
   When "addComplete and addFailed"  can access "flushContext.asyncAddArgList", `singleThreadExecutorForWrite` is no longer able to access the object `flushContext`. In method `doFlush`, `singleThreadExecutorForWrite` has already started holding another object `flushContext`: 
   
   ```java
   private void doFlush(){
         ......
         flushContext = FlushContext.newInstance();
         bytesSize = 0;
     }
   ```
   
   After `doFlush`, the thread `singleThreadExecutorForWrite` will not modify `flushContext` any more



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -143,6 +153,10 @@ public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor ordered
                                 DataSerializer<T> dataSerializer,
                                 int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
                                 boolean batchEnabled, TxnLogBufferedWriterMetricsStats metrics){
+        if (batchedWriteMaxRecords <= 1 && batchEnabled){
+            log.warn("The current maximum number of records per batch is set to 1. Disabling Batch will improve"

Review Comment:
   Already change it to this:
   
   ```
   if (metrics != null){
       log.warn("Transaction Log Buffered Writer with the metrics name beginning with {} has batching enabled"
               + " yet the maximum batch size was configured to less than or equal to 1 record, hence due to"
               + " performance reasons batching is disabled", metrics.getMetricsPrefix());
   } else {
       log.warn("Transaction Log Buffered Writer has batching enabled"
               + " yet the maximum batch size was configured to less than or equal to 1 record, hence due to"
               + " performance reasons batching is disabled");
   }
   ```
   
   the case `else` will be removed at the next PR



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -69,6 +70,10 @@
                     new Exception("Transaction log buffered write has closed")
             );
 
+    private static final AtomicReferenceFieldUpdater<TxnLogBufferedWriter, TxnLogBufferedWriter.State> STATE_UPDATER =

Review Comment:
   It has been changed to: Use `STATE_UPDATER ` to modify all state changes



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -112,10 +117,31 @@
 
     /** The main purpose of state maintenance is to prevent written after close. **/
     private volatile State state;
-    private static final AtomicReferenceFieldUpdater<TxnLogBufferedWriter, TxnLogBufferedWriter.State> STATE_UPDATER =
-            AtomicReferenceFieldUpdater
-                    .newUpdater(TxnLogBufferedWriter.class, TxnLogBufferedWriter.State.class, "state");
 
+    private final BookKeeperBatchedWriteCallback bookKeeperBatchedWriteCallback = new BookKeeperBatchedWriteCallback();
+
+    private final TxnLogBufferedWriterMetricsStats metrics;
+
+    /**
+     * In the {@link #asyncAddData}, exceptions may occur. To avoid losing the callback, use a variable to mark whether
+     * a callback needs to be compensated.
+     */
+    private boolean shouldCallBackWhenWriteFail;

Review Comment:
   Good suggestion. Already replaced with method-local variables



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterMetricsStats.java:
##########
@@ -118,84 +107,71 @@ public TxnLogBufferedWriterMetricsStats(String metricsPrefix, String[] labelName
 
         String recordsPerBatchMetricName =
                 String.format("%s_bufferedwriter_batch_record_count", metricsPrefix);
-        recordsPerBatchMetric = (Histogram) COLLECTOR_CACHE.computeIfAbsent(
-                recordsPerBatchMetricName,
-                k -> new Histogram.Builder()
+        this.recordsPerBatchMetric = new Histogram.Builder()

Review Comment:
   OK, already remove "this."



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -217,9 +224,9 @@ public void asyncAddData(T data, AddDataCallback callback, Object ctx){
                 // Avoid missing callback, do failed callback when error occur before add data to the array.
                 int recordsCountAfter = dataArray.size();
                 if (recordsCountAfter == recordsCountBeforeAdd){
-                    callback.addFailed(new ManagedLedgerException.ManagedLedgerFencedException(e), ctx);
+                    callback.addFailed(new ManagedLedgerInterceptException(e), ctx);

Review Comment:
   Log printing has been added



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