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/12 03:57:40 UTC

[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

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