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 10:22:42 UTC

[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

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