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/07/07 03:05:30 UTC

[GitHub] [pulsar] poorbarcode opened a new pull request, #16428: [improve][client] [PIP-165] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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

   Master Issue: #15370
   
   ### Motivation
   
   see #15370
   
   ### Modifications
   
   The first PR of PIP-160. Only contains `TxLogBufferedWriter`
   
   
   ### 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] poorbarcode commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,445 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        if (dataSerializer.getSerializedSize(data) >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));

Review Comment:
   Good suggestion



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            doFlush();
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    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);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray.clear();
+        this.asyncAddArgsList = null;
+        this.bytesSize = 0;
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
+     */
+    @Override
+    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        if (this.scheduledFuture != null) {

Review Comment:
   I have change the logic:  
   
   1. Force flush once before actual closed. 
   2. Change state of buffered writer to closed, avoid new requests coming in.
   
   > Why force flush once before actual closed ? 
   
   This ensures that the logic is always consistent, since it is possible that asynchronous timed refreshes are also being performed when closing.  If ML has been closed, pending requests will be triggered by ML to 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 pull request #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);

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 #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxLogBufferedWriter.java:
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+
+/***
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {

Review Comment:
   Good suggests. I have modified this class 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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);

Review Comment:
   > Should add error log here if encounter any exceptions, otherwise we will hard to investigate problems.
   
   Good idea.
   
   > And it is better to avoid using safeRun, for each task we will create a Runnable instance.
   
   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] congbobo184 commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;

Review Comment:
   final



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;

Review Comment:
   remove `<T>`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;

Review Comment:
   unuse remove



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static <T> FlushContext newInstance(){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            return flushContext;

Review Comment:
   ```suggestion
               return FLUSH_CONTEXT_RECYCLER.get();
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private ArrayList<AsyncAddArgs> asyncAddArgsList;

Review Comment:
   final



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;

Review Comment:
   seem not use 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;

Review Comment:
   remove `<T>`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{

Review Comment:
   remove `<T>`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static <T> FlushContext newInstance(){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            return flushContext;
+        }
+
+        public static <T> FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnLogBufferedWriter.TxnBatchedPositionImpl) {
+                TxnBatchedPositionImpl other = (TxnBatchedPositionImpl) o;
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }

Review Comment:
   ```suggestion
               if (o instanceof TxnBatchedPositionImpl other) {
                   return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
               }
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;

Review Comment:
   remove return 



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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) {

Review Comment:
   > Please check if the writer is closed, we shouldn't continue the callback process if the writer is closed.
   
   If Bookie returns a successful response, we should tell the caller that the execution succeeded, not that notice caller failed or ignored callback.
   
   - If callback is ignored, the caller may not receive the callback and be blocked.
   - If you tell the caller that the write failed, that make an inconsistent response.
   
   Both of these are wrong.
   



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;

Review Comment:
   If the list is reset to null for each batched write, why not just create a new list for each batched write. So that we don't need to maintain a list here?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;

Review Comment:
   ```suggestion
       private final int batchedWriteMaxDelayInMillis;
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;

Review Comment:
   And we should avoid creating a list for each batch? If there are 10k topics with 1ms max write delay, which means we will have 1000 * 10k ArrayList per second.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));

Review Comment:
   We can chose one thread instead of select a thread for each write operation.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            doFlush();
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    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);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray.clear();
+        this.asyncAddArgsList = null;
+        this.bytesSize = 0;
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
+     */
+    @Override
+    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        if (this.scheduledFuture != null) {

Review Comment:
   We should also failed the pending writes, clear the pending writes/



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);

Review Comment:
   The doTrigFlush method will do nothing?
   
   https://github.com/apache/pulsar/pull/16428/files#diff-cb4b7252e066d7063eca3fa5192eba481d0938449eb4ac22bb616ba75922a611R217-R219



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

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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{

Review Comment:
   Fixed.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{
+        RUNNING,
+        CLOSING,
+        CLOSED;
+    }
+
+    /***
+     * Instead origin param-callback for {@link #asyncAddData(Object, AddDataCallback, Object)}
+     * when {@link #batchEnabled} == false, Used for ByteBuf release which generated by {@link DataSerializer}.
+     */
+    public static class DisabledBatchCallback implements AsyncCallbacks.AddEntryCallback {
+
+        public static final DisabledBatchCallback INSTANCE = new DisabledBatchCallback();

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;

Review Comment:
   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 merged pull request #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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


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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static <T> FlushContext newInstance(){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            return flushContext;
+        }
+
+        public static <T> FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnLogBufferedWriter.TxnBatchedPositionImpl) {
+                TxnBatchedPositionImpl other = (TxnBatchedPositionImpl) o;
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }

Review Comment:
   Fixed.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,502 @@
+/**
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {

Review Comment:
   Please also create a test for the writer close behavior, check the state of the write, make sure the new operation will fail and the pending operations(haven't flush) will also fail.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,549 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** 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");
+
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(() -> trigFlush(false),
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.OPEN;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread. Callbacks are
+     * executed in strict write order,but after {@link #close()}, callbacks that fail by state check will execute
+     * earlier, and successful callbacks will not be affected.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    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);
+            }
+            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.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(final boolean force){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // 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();
+        }
+    }
+
+    /**
+     * 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.
+     */
+    @Override
+    public void close() {
+        // If disabled batch feature, there is no closing state.
+        if (!batchEnabled) {
+            STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
+            return;
+        }
+        // Prevent the reentrant.
+        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 (scheduledFuture != null && !scheduledFuture.isCancelled() && !scheduledFuture.isDone()) {
+                if (this.scheduledFuture.cancel(false)){
+                    this.state = State.CLOSED;
+                }
+            }
+            // Cancel task failure, The state will stay at CLOSING.
+            log.error("Cancel task that schedule at fixed rate trig flush failure. The state will stay at CLOSING."
+                    + " managedLedger: " + managedLedger.getName());
+        });

Review Comment:
   If the scheduledFuture already cancelled, we don't need this log.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,549 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** 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");
+
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(() -> trigFlush(false),
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.OPEN;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread. Callbacks are
+     * executed in strict write order,but after {@link #close()}, callbacks that fail by state check will execute
+     * earlier, and successful callbacks will not be affected.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    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);
+            }
+            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.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(final boolean force){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // 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();
+        }
+    }
+
+    /**
+     * 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.
+     */
+    @Override
+    public void close() {
+        // If disabled batch feature, there is no closing state.
+        if (!batchEnabled) {
+            STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
+            return;
+        }
+        // Prevent the reentrant.
+        if (!STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSING)){
+            // Other thread also calling "close()".
+            return;
+        }

Review Comment:
   Add some warning log here 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 pull request #16428: [improve] [transaction] [PIP-165] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] liangyepianzhou commented on a diff in pull request #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxLogBufferedWriter.java:
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+
+/***
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {

Review Comment:
   ```suggestion
   public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxLogBufferedWriter.java:
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+
+/***
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                               boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> asyncAddData0(data, callback, ctx));
+    }
+
+    private void asyncAddData0(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){

Review Comment:
   Please do not use 0 in the method name. 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxLogBufferedWriter.java:
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+
+/***
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                               boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> asyncAddData0(data, callback, ctx));
+    }
+
+    private void asyncAddData0(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);
+        // trig flush.
+        trigFlush0();
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), this::trigFlush0);
+    }
+
+    private void trigFlush0(){

Review Comment:
   Just like above.



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

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

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


[GitHub] [pulsar] congbobo184 commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);
+        // Create callback.
+        ArrayList<Integer> callbackCtxList = new ArrayList<>();
+        LinkedHashMap<PositionImpl, ArrayList<Position>> callbackPositions =
+                new LinkedHashMap<PositionImpl, ArrayList<Position>>();
+        AsyncCallbacks.AddEntryCallback callback = new AsyncCallbacks.AddEntryCallback(){
+            @Override
+            public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+                if (callbackCtxList.contains(Integer.valueOf(String.valueOf(ctx)))){
+                    return;
+                }
+                callbackCtxList.add((int)ctx);
+                PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId());
+                callbackPositions.computeIfAbsent(lightPosition, p -> new ArrayList<>());
+                callbackPositions.get(lightPosition).add(position);
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+            }
+        };
+        // Loop write data.  Holds variable dataArrayProvided just for release.
+        List<ByteBuf> dataArrayProvided = new ArrayList<>();
+        int cmdAddExecutedCount = 100000;
+        for (int i = 0; i < cmdAddExecutedCount; i++){
+            ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(8);
+            byteBuf.writeInt(i);
+            dataArrayProvided.add(byteBuf);
+            txnLogBufferedWriter.asyncAddData(byteBuf, callback, i);
+        }
+        // Wait for all cmd-write finish.
+        Awaitility.await().atMost(20, TimeUnit.SECONDS).until(() -> callbackCtxList.size() == cmdAddExecutedCount);

Review Comment:
   it's to long to wait a test



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,445 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        if (dataSerializer.getSerializedSize(data) >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));

Review Comment:
   move this logical to`internalAsyncAddData ` is better, otherwise, it will `dataSerializer.getSerializedSize(data)` twice. if the data is bigger than the max entry size, we should return exception



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);
+        // Create callback.
+        ArrayList<Integer> callbackCtxList = new ArrayList<>();
+        LinkedHashMap<PositionImpl, ArrayList<Position>> callbackPositions =
+                new LinkedHashMap<PositionImpl, ArrayList<Position>>();
+        AsyncCallbacks.AddEntryCallback callback = new AsyncCallbacks.AddEntryCallback(){
+            @Override
+            public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+                if (callbackCtxList.contains(Integer.valueOf(String.valueOf(ctx)))){
+                    return;
+                }
+                callbackCtxList.add((int)ctx);
+                PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId());
+                callbackPositions.computeIfAbsent(lightPosition, p -> new ArrayList<>());
+                callbackPositions.get(lightPosition).add(position);
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+            }
+        };
+        // Loop write data.  Holds variable dataArrayProvided just for release.
+        List<ByteBuf> dataArrayProvided = new ArrayList<>();
+        int cmdAddExecutedCount = 100000;

Review Comment:
   don't test so much entry



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);

Review Comment:
   every condition needs to be tested. In this case are time, data number, data 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 #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = -100;
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);
+        // trig flush.
+        doTrigFlush();
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), this::doTrigFlush);
+    }
+
+    private void doTrigFlush(){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    private void doFlush(){
+        // Combine data.
+        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
+        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.dataArray, this.asyncAddArgsList);

Review Comment:
   In `addComplete(Position position, ByteBuf byteBuf, Object ctx)` we return null param-byteBuf. Maybe callback implementation needs the data(T) in the future. 
   
   Does not need it right now, so delete 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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);
+        // Create callback.
+        ArrayList<Integer> callbackCtxList = new ArrayList<>();
+        LinkedHashMap<PositionImpl, ArrayList<Position>> callbackPositions =
+                new LinkedHashMap<PositionImpl, ArrayList<Position>>();
+        AsyncCallbacks.AddEntryCallback callback = new AsyncCallbacks.AddEntryCallback(){
+            @Override
+            public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+                if (callbackCtxList.contains(Integer.valueOf(String.valueOf(ctx)))){
+                    return;
+                }
+                callbackCtxList.add((int)ctx);
+                PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId());
+                callbackPositions.computeIfAbsent(lightPosition, p -> new ArrayList<>());
+                callbackPositions.get(lightPosition).add(position);
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+            }
+        };
+        // Loop write data.  Holds variable dataArrayProvided just for release.
+        List<ByteBuf> dataArrayProvided = new ArrayList<>();
+        int cmdAddExecutedCount = 100000;
+        for (int i = 0; i < cmdAddExecutedCount; i++){
+            ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(8);
+            byteBuf.writeInt(i);
+            dataArrayProvided.add(byteBuf);
+            txnLogBufferedWriter.asyncAddData(byteBuf, callback, i);
+        }
+        // Wait for all cmd-write finish.
+        Awaitility.await().atMost(20, TimeUnit.SECONDS).until(() -> callbackCtxList.size() == cmdAddExecutedCount);

Review Comment:
   Good suggestion



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;

Review Comment:
   Already fixed.
   
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            doFlush();
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    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);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray.clear();
+        this.asyncAddArgsList = null;
+        this.bytesSize = 0;
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
+     */
+    @Override
+    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);

Review Comment:
   @poorbarcode, I think this one is not fixed yet?



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

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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on pull request #16428: [improve][client] [PIP-165] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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

   @liangyepianzhou @congbobo184 Could you take a look


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){

Review Comment:
   Fixed.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            doFlush();
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    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);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray.clear();
+        this.asyncAddArgsList = null;
+        this.bytesSize = 0;
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
+     */
+    @Override
+    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,

Review Comment:
   Good suggestion



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{
+        RUNNING,

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,331 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);
+        // Create callback.
+        ArrayList<Integer> callbackCtxList = new ArrayList<>();
+        LinkedHashMap<PositionImpl, ArrayList<Position>> callbackPositions =
+                new LinkedHashMap<PositionImpl, ArrayList<Position>>();
+        TxnLogBufferedWriter.AddDataCallback callback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object ctx) {
+                if (callbackCtxList.contains(Integer.valueOf(String.valueOf(ctx)))){
+                    return;
+                }
+                callbackCtxList.add((int)ctx);
+                PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId());
+                callbackPositions.computeIfAbsent(lightPosition, p -> new ArrayList<>());
+                callbackPositions.get(lightPosition).add(position);
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+            }
+        };
+        // Loop write data.  Holds variable dataArrayProvided just for release.
+        List<ByteBuf> dataArrayProvided = new ArrayList<>();
+        int cmdAddExecutedCount = 5000;
+        for (int i = 0; i < cmdAddExecutedCount; i++){
+            ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(8);
+            byteBuf.writeInt(i);
+            dataArrayProvided.add(byteBuf);
+            txnLogBufferedWriter.asyncAddData(byteBuf, callback, i);
+        }
+        // Wait for all cmd-write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> callbackCtxList.size() == cmdAddExecutedCount);
+        // Release data provided.
+        for (ByteBuf byteBuf : dataArrayProvided){
+            byteBuf.release();
+        }
+        // Assert callback ctx correct.
+        Assert.assertEquals(callbackCtxList.size(), cmdAddExecutedCount);
+        for (int ctxIndex = 0; ctxIndex < cmdAddExecutedCount; ctxIndex++){
+            Assert.assertEquals(callbackCtxList.get(ctxIndex).intValue(), ctxIndex);
+        }
+        // Assert callback positions correct.
+        Assert.assertEquals(callbackPositions.values().stream().flatMap(l -> l.stream()).count(), cmdAddExecutedCount);
+        Iterator<ArrayList<Position>> callbackPositionIterator = callbackPositions.values().iterator();
+        for (int batchedEntryIndex = 0; batchedEntryIndex < stringBatchedEntryDataList.size(); batchedEntryIndex++){
+            String stringBatchedEntryData = stringBatchedEntryDataList.get(batchedEntryIndex);
+            String[] entryDataArray = stringBatchedEntryData.split(",");
+            ArrayList<Position> innerPositions = callbackPositionIterator.next();
+            int batchSize = entryDataArray.length;
+            for(int i = 0; i < entryDataArray.length; i++){
+                TxnLogBufferedWriter.TxnBatchedPositionImpl innerPosition =
+                        (TxnLogBufferedWriter.TxnBatchedPositionImpl) innerPositions.get(i);
+                Assert.assertEquals(innerPosition.getBatchSize(), batchSize);
+                Assert.assertEquals(innerPosition.getBatchIndex(), i);
+            }
+        }
+        // Assert content correct.
+        int batchedEntryIndex = 0;
+        Iterator<PositionImpl> expectedBatchedPositionIterator = callbackPositions.keySet().iterator();
+        while (managedCursor.hasMoreEntries()) {
+            List<Entry> entries = managedCursor.readEntries(1);
+            if (entries == null || entries.isEmpty()) {
+                continue;
+            }
+            for (int m = 0; m < entries.size(); m++) {
+                String stringBatchedEntryContent = stringBatchedEntryDataList.get(batchedEntryIndex);
+                Entry entry = entries.get(m);
+                ByteBuf entryByteBuf = entry.getDataBuffer();
+                entryByteBuf.skipBytes(4);
+                // Assert entry content correct.
+                byte[] entryContentBytes = new byte[entryByteBuf.readableBytes()];
+                entryByteBuf.readBytes(entryContentBytes);
+                String entryContentString = new String(entryContentBytes, Charset.defaultCharset());
+                Assert.assertEquals(entryContentString, stringBatchedEntryContent);
+                // Assert position correct.
+                PositionImpl expectPosition = expectedBatchedPositionIterator.next();
+                Assert.assertEquals(entry.getLedgerId(), expectPosition.getLedgerId());
+                Assert.assertEquals(entry.getEntryId(), expectPosition.getEntryId());
+                entry.release();
+                batchedEntryIndex++;
+            }
+        }
+        Assert.assertEquals(batchedEntryIndex, stringBatchedEntryDataList.size());
+        // cleanup.
+        txnLogBufferedWriter.close();
+        managedLedger.close();
+        scheduledExecutorService.shutdown();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * Test main process when disabled batch feature.
+     */
+    @Test
+    public void testDisabled() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        // Create TxLogBufferedWriter.
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, null, null,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>() {
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return 0;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                return null;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, false);
+        // Create callback.
+        CompletableFuture<Pair<Position, Object>> future = new CompletableFuture<>();
+        TxnLogBufferedWriter.AddDataCallback callback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object ctx) {
+                future.complete(Pair.of(position, ctx));
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+                future.completeExceptionally(exception);
+            }
+        };
+        // Async add data
+        ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(8);
+        byteBuf.writeInt(1);
+        txnLogBufferedWriter.asyncAddData(byteBuf, callback, 1);
+        // Wait add finish.
+        Pair<Position, Object> pair = future.get(2, TimeUnit.SECONDS);
+        // Assert callback ctx correct.
+        Assert.assertEquals(pair.getRight(), 1);
+        // Assert read entries correct.
+        List<Entry> entries = managedCursor.readEntriesOrWait(1);
+        Assert.assertEquals(entries.size(), 1);
+        Entry entry = entries.get(0);
+        Assert.assertEquals(entry.getLedgerId(), pair.getLeft().getLedgerId());
+        Assert.assertEquals(entry.getEntryId(), pair.getLeft().getEntryId());
+        Assert.assertEquals(entry.getDataBuffer().readInt(), 1);
+        entry.release();
+        // cleanup.
+        txnLogBufferedWriter.close();
+        managedLedger.close();
+    }
+
+    /**
+     * Adjustable thresholds: trigger BookKeeper-write when reaching any one of the following conditions
+     *     Max size (bytes)
+     *     Max records count
+     *     Max delay time
+     * Tests these three thresholds.
+     */
+    @Test
+    public void testFlushThresholds() throws Exception{
+        // Create components.
+        ManagedLedger managedLedger = Mockito.mock(ManagedLedger.class);
+        Mockito.when(managedLedger.getName()).thenReturn("-");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        TxnLogBufferedWriter.DataSerializer<Integer> serializer = new TxnLogBufferedWriter.DataSerializer<Integer>(){
+            @Override
+            public int getSerializedSize(Integer data) {
+                return 4;
+            }
+            @Override
+            public ByteBuf serialize(Integer data) {
+                return null;
+            }
+            @Override
+            public ByteBuf serialize(ArrayList<Integer> dataArray) {
+                int sum = CollectionUtils.isEmpty(dataArray) ? 0 : dataArray.stream().reduce((a, b) -> a+b).get();
+                ByteBuf byteBuf = Unpooled.buffer(4);
+                byteBuf.writeInt(sum);
+                return byteBuf;
+            }
+        };
+        List<Integer> flushedDataList = new ArrayList<>();
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                ByteBuf byteBuf = (ByteBuf)invocation.getArguments()[0];
+                byteBuf.skipBytes(4);
+                flushedDataList.add(byteBuf.readInt());
+                AsyncCallbacks.AddEntryCallback callback =
+                        (AsyncCallbacks.AddEntryCallback) invocation.getArguments()[1];
+                callback.addComplete(PositionImpl.get(1,1), byteBuf,
+                        invocation.getArguments()[2]);
+                return null;
+            }
+        }).when(managedLedger).asyncAddEntry(Mockito.any(ByteBuf.class), Mockito.any(), Mockito.any());
+
+        TxnLogBufferedWriter txnLogBufferedWriter = new TxnLogBufferedWriter<>(managedLedger, orderedExecutor,
+                scheduledExecutorService, serializer, 32, 1024 * 4, 100, true);
+        TxnLogBufferedWriter.AddDataCallback callback = Mockito.mock(TxnLogBufferedWriter.AddDataCallback.class);
+        // Test threshold: writeMaxDelayInMillis.
+        txnLogBufferedWriter.asyncAddData(100, callback, 100);

Review Comment:
   No, in the current case: the param-context argument is meaningless; it can be any 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] codelipenghui commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();

Review Comment:
   Same as the above comment.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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) {

Review Comment:
   Please check if the writer is closed, we shouldn't continue the callback process if the writer is closed.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);

Review Comment:
   We should check if the `scheduledFuture` is able to cancel first? If we got an exception here, the state will stay at CLOSING state.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;

Review Comment:
   missed recycle() method for this class?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){

Review Comment:
   ```suggestion
           private static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){

Review Comment:
   ```suggestion
           private static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);

Review Comment:
   Should add error log here if encounter any exceptions, otherwise we will hard to investigate problems.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{

Review Comment:
   ```suggestion
       private static class FlushContext{
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){

Review Comment:
   ```suggestion
           private static FlushContext newInstance(){
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);

Review Comment:
   We should complete the pending write with exceptions not flush them out? It looks like we closed the writer, but the writer is still working (not really closed)



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){

Review Comment:
   ```suggestion
           private FlushContext(Recycler.Handle<FlushContext> handle){
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {

Review Comment:
   ```suggestion
       private static class AsyncAddArgs {
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));

Review Comment:
   Why need to carry the `byteBuf` to the context?
   We can reuse the FlushContext? It's expensive to create a triple instance for each operation.



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,341 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {

Review Comment:
   Please also test the bookkeeper failure case 
   
   Here is an example https://github.com/apache/pulsar/blob/master/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java#L1265
   
   to make sure the failure handling of the writer is correct.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{
+        RUNNING,
+        CLOSING,
+        CLOSED;
+    }
+
+    /***
+     * Instead origin param-callback for {@link #asyncAddData(Object, AddDataCallback, Object)}
+     * when {@link #batchEnabled} == false, Used for ByteBuf release which generated by {@link DataSerializer}.
+     */
+    public static class DisabledBatchCallback implements AsyncCallbacks.AddEntryCallback {

Review Comment:
   ```suggestion
       private static class DisabledBatchCallback implements AsyncCallbacks.AddEntryCallback {
   ```



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);

Review Comment:
   Yes, it's good to have a data provider.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{
+        RUNNING,

Review Comment:
   I suggest using OPEN. The `RUNNING` state sounds a little bit confusing, it can easily be interpreted as the writer are processing some write operations for now.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {

Review Comment:
   If this one intended to be exposed to the outside, please move out from this class, just create a separate public class.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{
+        RUNNING,
+        CLOSING,
+        CLOSED;
+    }
+
+    /***
+     * Instead origin param-callback for {@link #asyncAddData(Object, AddDataCallback, Object)}
+     * when {@link #batchEnabled} == false, Used for ByteBuf release which generated by {@link DataSerializer}.
+     */
+    public static class DisabledBatchCallback implements AsyncCallbacks.AddEntryCallback {
+
+        public static final DisabledBatchCallback INSTANCE = new DisabledBatchCallback();

Review Comment:
   ```suggestion
           private static final DisabledBatchCallback INSTANCE = new DisabledBatchCallback();
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{

Review Comment:
   ```suggestion
       private enum State{
   ```



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);

Review Comment:
   And it is better to avoid using safeRun, for each task we will create a Runnable 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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,341 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {

Review Comment:
   > Yes, it's good to have a data provider.
   
   Good idea. Already fixed.
   
   > Please also test the bookkeeper failure case. to make sure the failure handling of the writer is correct.
   
   I have rewritten the unit tests to cover more test cases.



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

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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,502 @@
+/**
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {

Review Comment:
   > Please also create a test for the writer close behavior, check the state of the write, make sure the new operation will fail and the pending operations(haven't flush) will also fail.
   
   Yes, it is data provider case 5-1 & 5-2 ( `mainProcessCasesProvider[11-12]` )



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));

Review Comment:
   > Why need to carry the byteBuf to the context?
   
   This variable `byteBuf` was produced by `DataSerializer.serialize(T t)`, so we have to release it after the write is complete.
   
   > We can reuse the FlushContext? It's expensive to create a triple instance for each operation.
   
   Good idea. I have used `AsyncAddArgs` instead `Triple`.



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");

Review Comment:
   @congbobo184 Could you please provide more details? Why the `ManagedLedgerFencedException` 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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            doFlush();
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    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);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray.clear();
+        this.asyncAddArgsList = null;
+        this.bytesSize = 0;
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
+     */
+    @Override
+    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        if (this.scheduledFuture != null) {

Review Comment:
   Good suggestion.



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);

Review Comment:
   Yes. I will fix 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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,502 @@
+/**
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {

Review Comment:
   > Please also create a test for the writer close behavior, check the state of the write, make sure the new operation will fail and the pending operations(haven't flush) will also fail.
   
   Yes, it is data provider case 5-1 ( `mainProcessCasesProvider[12]` )



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

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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] poorbarcode commented on a diff in pull request #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = -100;
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);

Review Comment:
   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 #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }

Review Comment:
   Just for tests, I have delete it.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }

Review Comment:
   Just for tests, I have 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 pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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

   /pulsarbot run-failure-checks


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

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

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


[GitHub] [pulsar] congbobo184 commented on a diff in pull request #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = -100;
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);

Review Comment:
   if this bytesSize is bigger than batchedWriteMaxSize, we should return exception directly



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = -100;
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);
+        // trig flush.
+        doTrigFlush();
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), this::doTrigFlush);
+    }
+
+    private void doTrigFlush(){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    private void doFlush(){
+        // Combine data.
+        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
+        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.dataArray, this.asyncAddArgsList);

Review Comment:
   why FlushContext should use  `dataArray`



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }

Review Comment:
   where need this construction method



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = -100;
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);
+        // trig flush.
+        doTrigFlush();
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), this::doTrigFlush);
+    }
+
+    private void doTrigFlush(){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    private void doFlush(){
+        // Combine data.
+        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
+        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.dataArray, this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray = null;

Review Comment:
   may be we can clear the 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] poorbarcode commented on a diff in pull request #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,450 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = -100;
+
+    public static final int BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);
+        // trig flush.
+        doTrigFlush();
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), this::doTrigFlush);
+    }
+
+    private void doTrigFlush(){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    private void doFlush(){
+        // Combine data.
+        ByteBuf prefix = PulsarByteBufAllocator.DEFAULT.buffer(4);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER);
+        prefix.writeChar(BATCHED_ENTRY_DATA_PREFIX_VERSION);
+        ByteBuf actualContent = this.dataSerializer.serialize(this.dataArray);
+        ByteBuf pairByteBuf = Unpooled.wrappedUnmodifiableBuffer(prefix, actualContent);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.dataArray, this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray = null;

Review Comment:
   > may be we can clear the list
   
   No, even if have already called `managedLedger.asyncAdd`, still need to use the data in the 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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {
+
+        /** The data length of current batch. **/
+        @Getter
+        private final int batchSize;
+
+        /** The position of current batch. **/
+        @Getter
+        private final int batchIndex;
+
+        public TxnBatchedPositionImpl(Position position, int batchSize, int batchIndex, long[] ackSet){
+            super(position.getLedgerId(), position.getEntryId(), ackSet);
+            this.batchIndex = batchIndex;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (o instanceof TxnBatchedPositionImpl other) {
+                return super.equals(o) && batchSize == other.batchSize && batchIndex == other.batchIndex;
+            }
+            return false;
+
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), batchSize, batchIndex);
+        }
+    }
+
+    interface AddDataCallback {
+
+        void addComplete(Position position, Object context);
+
+        void addFailed(ManagedLedgerException exception, Object ctx);
+    }
+
+    public enum State{
+        RUNNING,
+        CLOSING,
+        CLOSED;
+    }
+
+    /***
+     * Instead origin param-callback for {@link #asyncAddData(Object, AddDataCallback, Object)}
+     * when {@link #batchEnabled} == false, Used for ByteBuf release which generated by {@link DataSerializer}.
+     */
+    public static class DisabledBatchCallback implements AsyncCallbacks.AddEntryCallback {

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static FlushContext newInstance(){
+            return FLUSH_CONTEXT_RECYCLER.get();
+        }
+
+        public static FlushContext newInstance(int asyncAddArgsListInitialCapacity){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            flushContext.asyncAddArgsList.ensureCapacity(asyncAddArgsListInitialCapacity);
+            return flushContext;
+        }
+
+        public void recycle(){
+            this.asyncAddArgsList.clear();
+            this.handle.recycle(this);
+        }
+    }
+
+    /***
+     * The difference with {@link PositionImpl} is that there are two more parameters:
+     * {@link #batchSize}, {@link #batchIndex}.
+     */
+    public static class TxnBatchedPositionImpl extends PositionImpl {

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);

Review Comment:
   Good suggestion



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,251 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, AsyncCallbacks.AddEntryCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);
+        // Create callback.
+        ArrayList<Integer> callbackCtxList = new ArrayList<>();
+        LinkedHashMap<PositionImpl, ArrayList<Position>> callbackPositions =
+                new LinkedHashMap<PositionImpl, ArrayList<Position>>();
+        AsyncCallbacks.AddEntryCallback callback = new AsyncCallbacks.AddEntryCallback(){
+            @Override
+            public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+                if (callbackCtxList.contains(Integer.valueOf(String.valueOf(ctx)))){
+                    return;
+                }
+                callbackCtxList.add((int)ctx);
+                PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId());
+                callbackPositions.computeIfAbsent(lightPosition, p -> new ArrayList<>());
+                callbackPositions.get(lightPosition).add(position);
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+            }
+        };
+        // Loop write data.  Holds variable dataArrayProvided just for release.
+        List<ByteBuf> dataArrayProvided = new ArrayList<>();
+        int cmdAddExecutedCount = 100000;

Review Comment:
   Good suggestion



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;

Review Comment:
   Fixed.



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){
+            this.handle = handle;
+            this.asyncAddArgsList = new ArrayList<>(8);
+        }
+
+        public static <T> FlushContext newInstance(){
+            FlushContext flushContext = FLUSH_CONTEXT_RECYCLER.get();
+            return flushContext;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Txn buffered writer for transaction log batch

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


##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,502 @@
+/**
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.metadata.api.MetadataStoreException;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {

Review Comment:
   > Please also create a test for the writer close behavior, check the state of the write, make sure the new operation will fail and the pending operations(haven't flush) will also fail.
   
   Yes, it is data provider case 5-1



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,549 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** 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");
+
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(() -> trigFlush(false),
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.OPEN;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread. Callbacks are
+     * executed in strict write order,but after {@link #close()}, callbacks that fail by state check will execute
+     * earlier, and successful callbacks will not be affected.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            if (state == State.CLOSING || state == State.CLOSED){
+                callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+                return;
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE,
+                    AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis(), byteBuf));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    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);
+            }
+            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.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        this.flushContext.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(final boolean force){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(force));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // 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();
+        }
+    }
+
+    /**
+     * 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.
+     */
+    @Override
+    public void close() {
+        // If disabled batch feature, there is no closing state.
+        if (!batchEnabled) {
+            STATE_UPDATER.compareAndSet(this, State.OPEN, State.CLOSED);
+            return;
+        }
+        // Prevent the reentrant.
+        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 (scheduledFuture != null && !scheduledFuture.isCancelled() && !scheduledFuture.isDone()) {
+                if (this.scheduledFuture.cancel(false)){
+                    this.state = State.CLOSED;
+                }
+            }
+            // Cancel task failure, The state will stay at CLOSING.
+            log.error("Cancel task that schedule at fixed rate trig flush failure. The state will stay at CLOSING."
+                    + " managedLedger: " + managedLedger.getName());
+        });

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 #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxLogBufferedWriter.java:
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+
+/***
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                               boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> asyncAddData0(data, callback, ctx));
+    }
+
+    private void asyncAddData0(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += dataSerializer.getSerializedSize(data);
+        // trig flush.
+        trigFlush0();
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), this::trigFlush0);
+    }
+
+    private void trigFlush0(){

Review Comment:
   OK. I have modified this method name to "doTrigFlush"



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

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 #16428: [improve] [transaction] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxLogBufferedWriter.java:
##########
@@ -0,0 +1,439 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+
+/***
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               boolean batchEnabled){
+        this(managedLedger, orderedExecutor, scheduledExecutorService, dataSerializer,
+                512, 1024 * 1024 * 4, 1, batchEnabled);
+    }
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                               ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                               int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                               boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)} when
+     *                 add complete.
+     *                 Note: When {@link #batchEnabled} == false, the param-byteBuf of {@param callback} will be null.
+     *                 Will call {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> asyncAddData0(data, callback, ctx));
+    }
+
+    private void asyncAddData0(T data, AsyncCallbacks.AddEntryCallback callback, Object ctx){

Review Comment:
   OK. I have modified this method name to "internalAsyncAddData"



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);

Review Comment:
   > We should complete the pending write with exceptions not flush them out? It looks like we closed the writer, but the writer is still working (not really closed).
   
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,483 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException.ManagedLedgerFencedException(
+                    new Exception("Transaction log buffered write has closed")
+            );
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private final ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+        this.state = State.RUNNING;
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext flushContext = (FlushContext) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private final ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+        public FlushContext(Recycler.Handle<FlushContext> handle){

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;

Review Comment:
   > If the list is reset to null for each batched write, why not just create a new list for each batched write. 
   
   Already fixed.
   
   >  So that we don't need to maintain a list here? 
   
   Transaction Log Buffered Write must hold either List or FlushContext because append requests are required to a container. 
   
   > And we should avoid creating a list for each batch? If there are 10k topics with 1ms max write delay, which means we will have 1000 * 10k ArrayList per second.
   
   I've changed the container for Transaction Log Buffered Write cache requests to `FlushContext`, and `FlushContext holds` reference of `ArrayList`, `ArrayList` will be reused by `FlushContext.recycle`.
   



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, 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] congbobo184 commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");

Review Comment:
   It's better to return ManagedLedgerFencedException. 



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");

Review Comment:
   ```suggestion
               new ManagedLedgerException("Transaction log buffered writer has closed");
   ```



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

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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    /** All write operation will be executed on single thread. **/
+    private final ExecutorService singleThreadExecutorForWrite;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int batchedWriteMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will create a new one after each batched writes.
+     */
+    private FlushContext<AsyncAddArgs> flushContext;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /** The main purpose of state maintenance is to prevent written after close. **/
+    private volatile State state;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param batchedWriteMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int batchedWriteMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.singleThreadExecutorForWrite = orderedExecutor.chooseThread(
+                managedLedger.getName() == null ? UUID.randomUUID().toString() : managedLedger.getName());
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.batchedWriteMaxDelayInMillis = batchedWriteMaxDelayInMillis;
+        this.flushContext = FlushContext.newInstance();
+        this.dataArray = new ArrayList<>();
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    batchedWriteMaxDelayInMillis, batchedWriteMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (state == State.CLOSING || state == State.CLOSED){
+            callback.addFailed(BUFFERED_WRITER_CLOSED_EXCEPTION, ctx);
+            return;
+        }
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        singleThreadExecutorForWrite.execute(() -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        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);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        singleThreadExecutorForWrite.execute(() -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (flushContext.asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            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();
+            return;
+        }
+    }
+
+    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);
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+        // Clear buffers.
+        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<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
+                        batchIndex, ackSet);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> {
+                    asyncAddArgs.callback.addComplete(txnBatchedPosition, asyncAddArgs.ctx);
+                }).run();
+            }
+        } finally {
+            entryData.release();
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addFailed(ManagedLedgerException, Object)}.
+     */
+    @Override
+    public void addFailed(ManagedLedgerException exception, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            for (int i = 0; i < flushContext.asyncAddArgsList.size(); i++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(i);
+                // Because this task already running at ordered task, so just "run".
+                SafeRunnable.safeRun(() -> asyncAddArgs.callback.addFailed(exception, asyncAddArgs.ctx)).run();
+            }
+        } finally {
+            flushContext.recycle();
+        }
+    }
+
+    /**
+     * Release resources and executing tasks.
+     */
+    @Override
+    public void close() {
+        this.state = State.CLOSING;
+        singleThreadExecutorForWrite.execute(() -> {
+            if (!flushContext.asyncAddArgsList.isEmpty()) {
+                doTrigFlush(true);
+            }
+        });
+        if (this.scheduledFuture != null) {
+            this.scheduledFuture.cancel(false);
+        }
+        this.state = State.CLOSED;
+    }
+
+    /***
+     * Holds the parameters of {@link #asyncAddData} for each callback after batch write finished. This object is
+     * designed to be reusable, so the recycle mechanism is used。
+     */
+    @ToString
+    public static class AsyncAddArgs {
+
+        private static final Recycler<AsyncAddArgs> ASYNC_ADD_ARGS_RECYCLER = new Recycler<>() {
+            @Override
+            protected AsyncAddArgs newObject(Handle<AsyncAddArgs> handle) {
+                return new AsyncAddArgs(handle);
+            }
+        };
+
+        public static AsyncAddArgs newInstance(AddDataCallback callback, Object ctx, long addedTime){
+            AsyncAddArgs asyncAddArgs = ASYNC_ADD_ARGS_RECYCLER.get();
+            asyncAddArgs.callback = callback;
+            asyncAddArgs.ctx = ctx;
+            asyncAddArgs.addedTime = addedTime;
+            return asyncAddArgs;
+        }
+
+        private AsyncAddArgs(Recycler.Handle<AsyncAddArgs> handle){
+            this.handle = handle;
+        }
+
+        private final Recycler.Handle<AsyncAddArgs> handle;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private AddDataCallback callback;
+
+        /** Argument for {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private Object ctx;
+
+        /** Time of executed {@link #asyncAddData(Object, AddDataCallback, Object)}. **/
+        @Getter
+        private long addedTime;
+
+    }
+
+    /***
+     * The context for {@link ManagedLedger#asyncAddEntry(byte[], AsyncCallbacks.AddEntryCallback, Object)}, Holds the
+     * data array written in batches and callback parameters that need to be executed after batched write is complete.
+     */
+    public static class FlushContext<T>{
+
+        private static final Recycler<FlushContext> FLUSH_CONTEXT_RECYCLER = new Recycler<FlushContext>() {
+            @Override
+            protected FlushContext newObject(Handle<FlushContext> handle) {
+                return new FlushContext(handle);
+            }
+        };
+
+        private final Recycler.Handle<FlushContext> handle;
+
+        /** Callback parameters for current batch. **/
+        private ArrayList<AsyncAddArgs> asyncAddArgsList;

Review Comment:
   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 #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,489 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    private static final ManagedLedgerException BUFFERED_WRITER_CLOSED_EXCEPTION =
+            new ManagedLedgerException("Transaction log buffered write has closed");

Review Comment:
   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] liangyepianzhou commented on a diff in pull request #16428: [improve] [txn] [PIP-160] Make transactions work more efficiently by aggregation operation for transaction log and pending ack store

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


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriter.java:
##########
@@ -0,0 +1,453 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Recycler;
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Objects;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.Getter;
+import lombok.ToString;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.common.util.collections.BitSetRecyclable;
+
+/***
+ * See PIP-160: https://github.com/apache/pulsar/issues/15516.
+ * Buffer requests and flush to Managed Ledger. Transaction Log Store And Pending Ack Store will no longer write to
+ * Managed Ledger directly, Change to using this class to write Ledger data.
+ * Caches “write requests” for a certain number or a certain size of request data and then writes them to the
+ * Managed Ledger in one go. After Managed Ledger has written complete, responds to each request-caller. In this
+ * process, Managed Ledger doesn't care how many records(or what to be written) in the Entry, it just treats them as
+ * a single block of data.
+ * The first write-request by transaction components will take a long time to receive a response, because we must wait
+ * for subsequent requests to accumulate enough data to actually start writing to the Managed Ledger. To control the
+ * maximum latency, we will mark the first request time for each batch, and additional timing triggers writes.
+ * You can enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+ */
+@Slf4j
+public class TxnLogBufferedWriter<T> implements AsyncCallbacks.AddEntryCallback, Closeable {
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER = 0x0e01;
+
+    public static final short BATCHED_ENTRY_DATA_PREFIX_VERSION = 1;
+
+    /**
+     * Enable or disabled the batch feature, will use Managed Ledger directly and without batching when disabled.
+     */
+    private final boolean batchEnabled;
+
+    private final ManagedLedger managedLedger;
+
+    private final OrderedExecutor orderedExecutor;
+
+    private final ScheduledExecutorService scheduledExecutorService;
+
+    /** The serializer for the object which called by {@link #asyncAddData}. **/
+    private final DataSerializer<T> dataSerializer;
+
+    private ScheduledFuture<?> scheduledFuture;
+
+    /**
+     * Caches “write requests” for a certain for a certain number, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxRecords;
+
+    /**
+     * Caches “write requests” for a certain size of request data, if reach this threshold, will trig Bookie writes.
+     */
+    private final int batchedWriteMaxSize;
+
+    /** Maximum delay for writing to bookie for the earliest request in the batch. **/
+    private final int writeMaxDelayInMillis;
+
+    /** Data cached in the current batch. Will reset to null after each batched writes. **/
+    private ArrayList<T> dataArray;
+
+    /**
+     * Parameters of {@link #asyncAddData} cached in the current batch. Will reset to null after each batched writes.
+     */
+    private ArrayList<AsyncAddArgs> asyncAddArgsList;
+
+    /** Bytes size of data in current batch. Will reset to 0 after each batched writes. **/
+    private long bytesSize;
+
+    /**
+     * Constructor.
+     * @param dataSerializer The serializer for the object which called by {@link #asyncAddData}.
+     * @param batchedWriteMaxRecords Caches “write requests” for a certain number, if reach this threshold, will trig
+     *                               Bookie writes.
+     * @param batchedWriteMaxSize Caches “write requests” for a certain size of request data, if reach this threshold,
+     *                           will trig Bookie writes.
+     * @param writeMaxDelayInMillis Maximum delay for writing to bookie for the earliest request in the batch.
+     * @param batchEnabled Enable or disabled the batch feature, will use Managed Ledger directly and without batching
+     *                    when disabled.
+     */
+    public TxnLogBufferedWriter(ManagedLedger managedLedger, OrderedExecutor orderedExecutor,
+                                ScheduledExecutorService scheduledExecutorService, DataSerializer<T> dataSerializer,
+                                int batchedWriteMaxRecords, int batchedWriteMaxSize, int writeMaxDelayInMillis,
+                                boolean batchEnabled){
+        this.batchEnabled = batchEnabled;
+        this.managedLedger = managedLedger;
+        this.orderedExecutor = orderedExecutor;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.dataSerializer = dataSerializer;
+        this.batchedWriteMaxRecords = batchedWriteMaxRecords;
+        this.batchedWriteMaxSize = batchedWriteMaxSize;
+        this.writeMaxDelayInMillis = writeMaxDelayInMillis;
+        // scheduler task.
+        if (batchEnabled) {
+            this.scheduledFuture = this.scheduledExecutorService.scheduleAtFixedRate(this::trigFlush,
+                    writeMaxDelayInMillis, writeMaxDelayInMillis, TimeUnit.MICROSECONDS);
+        }
+    }
+
+    /**
+     * Append a new entry to the end of a managed ledger. All writes will be performed in the same thread.
+     * @param data data entry to be persisted.
+     * @param callback Will call {@link AddDataCallback#addComplete(Position, Object)} when
+     *                 add complete.
+     *                 Will call {@link AddDataCallback#addFailed(ManagedLedgerException, Object)} when
+     *                 add failure.
+     * @throws ManagedLedgerException
+     */
+    public void asyncAddData(T data, AddDataCallback callback, Object ctx){
+        if (!batchEnabled){
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> internalAsyncAddData(data, callback, ctx));
+    }
+
+    private void internalAsyncAddData(T data, AddDataCallback callback, Object ctx){
+        int len = dataSerializer.getSerializedSize(data);
+        if (len >= batchedWriteMaxSize){
+            doTrigFlush(true);
+            ByteBuf byteBuf = dataSerializer.serialize(data);
+            managedLedger.asyncAddEntry(byteBuf, DisabledBatchCallback.INSTANCE, Triple.of(byteBuf, callback, ctx));
+            return;
+        }
+        // Add data.
+        if (this.dataArray == null){
+            this.dataArray = new ArrayList<>();
+        }
+        this.dataArray.add(data);
+        // Add callback info.
+        AsyncAddArgs asyncAddArgs = AsyncAddArgs.newInstance(callback, ctx, System.currentTimeMillis());
+        if (this.asyncAddArgsList == null){
+            this.asyncAddArgsList = new ArrayList<>();
+        }
+        this.asyncAddArgsList.add(asyncAddArgs);
+        // Calculate bytes-size.
+        this.bytesSize += len;
+        // trig flush.
+        doTrigFlush(false);
+    }
+
+    /***
+     * The serializer for the object which called by {@link #asyncAddData}.
+     */
+    public interface DataSerializer<T>{
+
+        /**
+         * Calculate the number of bytes taken by {@param data} after serialization.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return The number of bytes taken after serialization.
+         */
+        int getSerializedSize(T data);
+
+        /**
+         * Serialize {@param data} to {@link ByteBuf}. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param data The object which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(T data);
+
+        /**
+         * Serialize {@param dataArray} to {@link ByteBuf}.. The returned ByteBuf will be release once after writing to
+         * Bookie complete, and if you still need to use the ByteBuf, should call {@link ByteBuf#retain()} in
+         * {@link #serialize(Object)} implementation.
+         * @param dataArray The objects which called by {@link #asyncAddData}.
+         * @return byte buf.
+         */
+        ByteBuf serialize(ArrayList<T> dataArray);
+
+    }
+
+    /**
+     * Trigger write to bookie once, If the conditions are not met, nothing will be done.
+     */
+    public void trigFlush(){
+        orderedExecutor.executeOrdered(managedLedger.getName(), () -> doTrigFlush(false));
+    }
+
+    private void doTrigFlush(boolean force){
+        if (asyncAddArgsList == null || asyncAddArgsList.isEmpty()) {
+            return;
+        }
+        if (force){
+            doFlush();
+            return;
+        }
+        AsyncAddArgs firstAsyncAddArgs = asyncAddArgsList.get(0);
+        if (System.currentTimeMillis() - firstAsyncAddArgs.addedTime > writeMaxDelayInMillis){
+            doFlush();
+            return;
+        }
+        if (this.asyncAddArgsList.size() >= batchedWriteMaxRecords){
+            doFlush();
+            return;
+        }
+        if (this.bytesSize >= batchedWriteMaxSize){
+            doFlush();
+            return;
+        }
+    }
+
+    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);
+        FlushContext<T> flushContext = FlushContext.newInstance(this.asyncAddArgsList);
+        // Clear buffers.
+        this.dataArray.clear();
+        this.asyncAddArgsList = null;
+        this.bytesSize = 0;
+        // Flush.
+        managedLedger.asyncAddEntry(pairByteBuf, this, flushContext);
+    }
+
+    /**
+     * see {@link AsyncCallbacks.AddEntryCallback#addComplete(Position, ByteBuf, Object)}.
+     */
+    @Override
+    public void addComplete(Position position, ByteBuf entryData, Object ctx) {
+        final FlushContext<T> flushContext = (FlushContext<T>) ctx;
+        try {
+            final int batchSize = flushContext.asyncAddArgsList.size();
+            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++) {
+                final AsyncAddArgs asyncAddArgs = flushContext.asyncAddArgsList.get(batchIndex);
+                BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create();
+                bitSetRecyclable.set(batchIndex);
+                long[] ackSet = bitSetRecyclable.toLongArray();
+                bitSetRecyclable.recycle();
+                final TxnBatchedPositionImpl txBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,

Review Comment:
   ```suggestion
                   final TxnBatchedPositionImpl txnBatchedPosition = new TxnBatchedPositionImpl(position, batchSize,
   ```



##########
pulsar-transaction/coordinator/src/test/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterTest.java:
##########
@@ -0,0 +1,331 @@
+/**
+ * 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.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.PreferHeapByteBufAllocator;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.ManagedLedger;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
+import org.apache.pulsar.transaction.coordinator.test.MockedBookKeeperTestCase;
+import org.awaitility.Awaitility;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@Slf4j
+public class TxnLogBufferedWriterTest extends MockedBookKeeperTestCase {
+
+    /**
+     * Tests all operations from write to callback, including
+     * {@link TxnLogBufferedWriter#asyncAddData(Object, TxnLogBufferedWriter.AddDataCallback, Object)}
+     * {@link TxnLogBufferedWriter#trigFlush()}
+     * and so on.
+     */
+    @Test
+    public void testMainProcess() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        // Create TxLogBufferedWriter.
+        ArrayList<String> stringBatchedEntryDataList = new ArrayList<>();
+        // Holds variable byteBufBatchedEntryDataList just for release.
+        ArrayList<ByteBuf> byteBufBatchedEntryDataList = new ArrayList<>();
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, orderedExecutor, scheduledExecutorService,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>(){
+
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return byteBuf.readableBytes();
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                StringBuilder stringBuilder = new StringBuilder();
+                                for (int i = 0; i < dataArray.size(); i++){
+                                    ByteBuf byteBuf = dataArray.get(i);
+                                    byteBuf.markReaderIndex();
+                                    stringBuilder.append(byteBuf.readInt());
+                                    if (i != dataArray.size() - 1){
+                                        stringBuilder.append(",");
+                                    }
+                                }
+                                String contentStr = stringBuilder.toString();
+                                stringBatchedEntryDataList.add(contentStr);
+                                byte[] bs = contentStr.getBytes(Charset.defaultCharset());
+                                ByteBuf content = PreferHeapByteBufAllocator.DEFAULT.buffer(bs.length);
+                                content.writeBytes(bs);
+                                byteBufBatchedEntryDataList.add(content);
+                                return content;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, true);
+        // Create callback.
+        ArrayList<Integer> callbackCtxList = new ArrayList<>();
+        LinkedHashMap<PositionImpl, ArrayList<Position>> callbackPositions =
+                new LinkedHashMap<PositionImpl, ArrayList<Position>>();
+        TxnLogBufferedWriter.AddDataCallback callback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object ctx) {
+                if (callbackCtxList.contains(Integer.valueOf(String.valueOf(ctx)))){
+                    return;
+                }
+                callbackCtxList.add((int)ctx);
+                PositionImpl lightPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId());
+                callbackPositions.computeIfAbsent(lightPosition, p -> new ArrayList<>());
+                callbackPositions.get(lightPosition).add(position);
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+            }
+        };
+        // Loop write data.  Holds variable dataArrayProvided just for release.
+        List<ByteBuf> dataArrayProvided = new ArrayList<>();
+        int cmdAddExecutedCount = 5000;
+        for (int i = 0; i < cmdAddExecutedCount; i++){
+            ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(8);
+            byteBuf.writeInt(i);
+            dataArrayProvided.add(byteBuf);
+            txnLogBufferedWriter.asyncAddData(byteBuf, callback, i);
+        }
+        // Wait for all cmd-write finish.
+        Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> callbackCtxList.size() == cmdAddExecutedCount);
+        // Release data provided.
+        for (ByteBuf byteBuf : dataArrayProvided){
+            byteBuf.release();
+        }
+        // Assert callback ctx correct.
+        Assert.assertEquals(callbackCtxList.size(), cmdAddExecutedCount);
+        for (int ctxIndex = 0; ctxIndex < cmdAddExecutedCount; ctxIndex++){
+            Assert.assertEquals(callbackCtxList.get(ctxIndex).intValue(), ctxIndex);
+        }
+        // Assert callback positions correct.
+        Assert.assertEquals(callbackPositions.values().stream().flatMap(l -> l.stream()).count(), cmdAddExecutedCount);
+        Iterator<ArrayList<Position>> callbackPositionIterator = callbackPositions.values().iterator();
+        for (int batchedEntryIndex = 0; batchedEntryIndex < stringBatchedEntryDataList.size(); batchedEntryIndex++){
+            String stringBatchedEntryData = stringBatchedEntryDataList.get(batchedEntryIndex);
+            String[] entryDataArray = stringBatchedEntryData.split(",");
+            ArrayList<Position> innerPositions = callbackPositionIterator.next();
+            int batchSize = entryDataArray.length;
+            for(int i = 0; i < entryDataArray.length; i++){
+                TxnLogBufferedWriter.TxnBatchedPositionImpl innerPosition =
+                        (TxnLogBufferedWriter.TxnBatchedPositionImpl) innerPositions.get(i);
+                Assert.assertEquals(innerPosition.getBatchSize(), batchSize);
+                Assert.assertEquals(innerPosition.getBatchIndex(), i);
+            }
+        }
+        // Assert content correct.
+        int batchedEntryIndex = 0;
+        Iterator<PositionImpl> expectedBatchedPositionIterator = callbackPositions.keySet().iterator();
+        while (managedCursor.hasMoreEntries()) {
+            List<Entry> entries = managedCursor.readEntries(1);
+            if (entries == null || entries.isEmpty()) {
+                continue;
+            }
+            for (int m = 0; m < entries.size(); m++) {
+                String stringBatchedEntryContent = stringBatchedEntryDataList.get(batchedEntryIndex);
+                Entry entry = entries.get(m);
+                ByteBuf entryByteBuf = entry.getDataBuffer();
+                entryByteBuf.skipBytes(4);
+                // Assert entry content correct.
+                byte[] entryContentBytes = new byte[entryByteBuf.readableBytes()];
+                entryByteBuf.readBytes(entryContentBytes);
+                String entryContentString = new String(entryContentBytes, Charset.defaultCharset());
+                Assert.assertEquals(entryContentString, stringBatchedEntryContent);
+                // Assert position correct.
+                PositionImpl expectPosition = expectedBatchedPositionIterator.next();
+                Assert.assertEquals(entry.getLedgerId(), expectPosition.getLedgerId());
+                Assert.assertEquals(entry.getEntryId(), expectPosition.getEntryId());
+                entry.release();
+                batchedEntryIndex++;
+            }
+        }
+        Assert.assertEquals(batchedEntryIndex, stringBatchedEntryDataList.size());
+        // cleanup.
+        txnLogBufferedWriter.close();
+        managedLedger.close();
+        scheduledExecutorService.shutdown();
+        orderedExecutor.shutdown();
+    }
+
+    /**
+     * Test main process when disabled batch feature.
+     */
+    @Test
+    public void testDisabled() throws Exception {
+        // Create components.
+        ManagedLedger managedLedger = factory.open("tx_test_ledger");
+        ManagedCursor managedCursor = managedLedger.openCursor("tx_test_cursor");
+        // Create TxLogBufferedWriter.
+        TxnLogBufferedWriter txnLogBufferedWriter =
+                new TxnLogBufferedWriter<ByteBuf>(managedLedger, null, null,
+                        new TxnLogBufferedWriter.DataSerializer<ByteBuf>() {
+                            @Override
+                            public int getSerializedSize(ByteBuf byteBuf) {
+                                return 0;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ByteBuf byteBuf) {
+                                return byteBuf;
+                            }
+
+                            @Override
+                            public ByteBuf serialize(ArrayList<ByteBuf> dataArray) {
+                                return null;
+                            }
+                        }, 512, 1024 * 1024 * 4, 1, false);
+        // Create callback.
+        CompletableFuture<Pair<Position, Object>> future = new CompletableFuture<>();
+        TxnLogBufferedWriter.AddDataCallback callback = new TxnLogBufferedWriter.AddDataCallback(){
+            @Override
+            public void addComplete(Position position, Object ctx) {
+                future.complete(Pair.of(position, ctx));
+            }
+            @Override
+            public void addFailed(ManagedLedgerException exception, Object ctx) {
+                future.completeExceptionally(exception);
+            }
+        };
+        // Async add data
+        ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(8);
+        byteBuf.writeInt(1);
+        txnLogBufferedWriter.asyncAddData(byteBuf, callback, 1);
+        // Wait add finish.
+        Pair<Position, Object> pair = future.get(2, TimeUnit.SECONDS);
+        // Assert callback ctx correct.
+        Assert.assertEquals(pair.getRight(), 1);
+        // Assert read entries correct.
+        List<Entry> entries = managedCursor.readEntriesOrWait(1);
+        Assert.assertEquals(entries.size(), 1);
+        Entry entry = entries.get(0);
+        Assert.assertEquals(entry.getLedgerId(), pair.getLeft().getLedgerId());
+        Assert.assertEquals(entry.getEntryId(), pair.getLeft().getEntryId());
+        Assert.assertEquals(entry.getDataBuffer().readInt(), 1);
+        entry.release();
+        // cleanup.
+        txnLogBufferedWriter.close();
+        managedLedger.close();
+    }
+
+    /**
+     * Adjustable thresholds: trigger BookKeeper-write when reaching any one of the following conditions
+     *     Max size (bytes)
+     *     Max records count
+     *     Max delay time
+     * Tests these three thresholds.
+     */
+    @Test
+    public void testFlushThresholds() throws Exception{
+        // Create components.
+        ManagedLedger managedLedger = Mockito.mock(ManagedLedger.class);
+        Mockito.when(managedLedger.getName()).thenReturn("-");
+        OrderedExecutor orderedExecutor =  OrderedExecutor.newBuilder()
+                .numThreads(5).name("tx-brokers-topic-workers").build();
+        ScheduledExecutorService scheduledExecutorService =
+                Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-stats-updater"));
+        TxnLogBufferedWriter.DataSerializer<Integer> serializer = new TxnLogBufferedWriter.DataSerializer<Integer>(){
+            @Override
+            public int getSerializedSize(Integer data) {
+                return 4;
+            }
+            @Override
+            public ByteBuf serialize(Integer data) {
+                return null;
+            }
+            @Override
+            public ByteBuf serialize(ArrayList<Integer> dataArray) {
+                int sum = CollectionUtils.isEmpty(dataArray) ? 0 : dataArray.stream().reduce((a, b) -> a+b).get();
+                ByteBuf byteBuf = Unpooled.buffer(4);
+                byteBuf.writeInt(sum);
+                return byteBuf;
+            }
+        };
+        List<Integer> flushedDataList = new ArrayList<>();
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                ByteBuf byteBuf = (ByteBuf)invocation.getArguments()[0];
+                byteBuf.skipBytes(4);
+                flushedDataList.add(byteBuf.readInt());
+                AsyncCallbacks.AddEntryCallback callback =
+                        (AsyncCallbacks.AddEntryCallback) invocation.getArguments()[1];
+                callback.addComplete(PositionImpl.get(1,1), byteBuf,
+                        invocation.getArguments()[2]);
+                return null;
+            }
+        }).when(managedLedger).asyncAddEntry(Mockito.any(ByteBuf.class), Mockito.any(), Mockito.any());
+
+        TxnLogBufferedWriter txnLogBufferedWriter = new TxnLogBufferedWriter<>(managedLedger, orderedExecutor,
+                scheduledExecutorService, serializer, 32, 1024 * 4, 100, true);
+        TxnLogBufferedWriter.AddDataCallback callback = Mockito.mock(TxnLogBufferedWriter.AddDataCallback.class);
+        // Test threshold: writeMaxDelayInMillis.
+        txnLogBufferedWriter.asyncAddData(100, callback, 100);

Review Comment:
   I don't understand why the ctx here is 100, does it only play a role in the log?



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

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

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