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/26 04:34:08 UTC

[GitHub] [pulsar] codelipenghui commented on a diff in pull request #16707: [improve][txn] PIP-160: Pending ack log store enables the batch feature

codelipenghui commented on code in PR #16707:
URL: https://github.com/apache/pulsar/pull/16707#discussion_r929508250


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -224,6 +224,9 @@ public class BrokerService implements Closeable {
     @Getter
     private final ScheduledExecutorService backlogQuotaChecker;
 
+    @Getter
+    private final ScheduledExecutorService transactionTimer;

Review Comment:
   Looks unused.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java:
##########
@@ -344,6 +347,8 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws
         this.backlogQuotaManager = new BacklogQuotaManager(pulsar);
         this.backlogQuotaChecker = Executors
                 .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-backlog-quota-checker"));
+        this.transactionTimer = Executors
+                .newSingleThreadScheduledExecutor(new DefaultThreadFactory("pulsar-backlog-quota-checker"));

Review Comment:
   Looks unused.



##########
pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStoreTest.java:
##########
@@ -0,0 +1,263 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.pendingack.impl;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.ManagedCursor;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.pulsar.broker.ServiceConfiguration;
+import org.apache.pulsar.broker.service.Subscription;
+import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.transaction.TransactionTestBase;
+import org.apache.pulsar.broker.transaction.util.LogIndexLagBackoff;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.api.proto.CommandAck;
+import org.apache.pulsar.common.api.proto.CommandSubscribe;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.transaction.coordinator.impl.TxnLogBufferedWriterConfig;
+import static org.mockito.Mockito.*;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.testcontainers.shaded.org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+@Slf4j
+@Test(groups = "broker")
+public class MLPendingAckStoreTest extends TransactionTestBase {
+
+    private PersistentSubscription persistentSubscriptionMock;
+
+    private ManagedCursor managedCursorMock;
+
+    private ExecutorService internalPinnedExecutor;
+
+    private int pendingAckLogIndexMinLag = 1;
+
+    @BeforeMethod
+    @Override
+    protected void setup() throws Exception {
+        setUpBase(1, 1, NAMESPACE1 + "/test", 0);
+        String topic = NAMESPACE1 + "/test-txn-topic";
+        admin.topics().createNonPartitionedTopic(topic);
+        PersistentTopic persistentTopic = (PersistentTopic) getPulsarServiceList().get(0).getBrokerService()
+                .getTopic(topic, false).get().get();
+        getPulsarServiceList().get(0).getConfig().setTransactionPendingAckLogIndexMinLag(pendingAckLogIndexMinLag);
+        CompletableFuture<Subscription> subscriptionFuture = persistentTopic .createSubscription("test",
+                        CommandSubscribe.InitialPosition.Earliest, false, null);
+        PersistentSubscription subscription = (PersistentSubscription) subscriptionFuture.get();
+        ManagedCursor managedCursor = subscription.getCursor();
+        this.managedCursorMock = spy(managedCursor);
+        this.persistentSubscriptionMock = spy(subscription);
+        when(this.persistentSubscriptionMock.getCursor()).thenReturn(managedCursorMock);
+        this.internalPinnedExecutor = this.persistentSubscriptionMock
+                .getTopic()
+                .getBrokerService()
+                .getPulsar()
+                .getTransactionExecutorProvider()
+                .getExecutor(this);
+    }
+
+    @AfterMethod
+    public void cleanup(){
+        super.internalCleanup();
+    }
+
+    private MLPendingAckStore createPendingAckStore(TxnLogBufferedWriterConfig txnLogBufferedWriterConfig)
+            throws Exception {
+        MLPendingAckStoreProvider mlPendingAckStoreProvider = new MLPendingAckStoreProvider();
+        ServiceConfiguration serviceConfiguration =
+                persistentSubscriptionMock.getTopic().getBrokerService().getPulsar().getConfiguration();
+        serviceConfiguration.setTransactionPendingAckBatchedWriteMaxRecords(
+                txnLogBufferedWriterConfig.getBatchedWriteMaxRecords()
+        );
+        serviceConfiguration.setTransactionPendingAckBatchedWriteMaxSize(
+                txnLogBufferedWriterConfig.getBatchedWriteMaxSize()
+        );
+        serviceConfiguration.setTransactionPendingAckBatchedWriteMaxDelayInMillis(
+                txnLogBufferedWriterConfig.getBatchedWriteMaxDelayInMillis()
+        );
+        serviceConfiguration.setTransactionPendingAckBatchedWriteEnabled(txnLogBufferedWriterConfig.isBatchEnabled());
+        return (MLPendingAckStore) mlPendingAckStoreProvider.newPendingAckStore(persistentSubscriptionMock).get();
+    }
+
+    /**
+     * Overridden cases:
+     *   1. Batched write and replay with batched feature.
+     *   1. Non-batched write and replay without batched feature
+     *   1. Batched write and replay without batched feature.
+     *   1. Non-batched write and replay with batched feature.
+     */
+    @DataProvider(name = "mainProcessArgs")
+    public Object[][] mainProcessArgsProvider(){
+        Object[][] args = new Object[4][];
+        args[0] = new Object[]{true, true};
+        args[1] = new Object[]{false, false};
+        args[2] = new Object[]{true, false};
+        args[3] = new Object[]{false, true};
+        return args;
+    }
+
+    /**
+     * This method executed the following steps of validation:
+     *   1. Write some data, verify indexes build correct after write.
+     *   2. Replay data that has been written, verify indexes build correct after replay.
+     *   3. Verify that position deletion is in sync with {@link PersistentSubscription}.
+     * @param writeWithBatch Whether to enable batch feature when writing data.
+     * @param readWithBatch Whether to enable batch feature when replay.
+     */
+    @Test(dataProvider = "mainProcessArgs")
+    public void testMainProcess(boolean writeWithBatch, boolean readWithBatch) throws Exception {
+        // Write some data.
+        TxnLogBufferedWriterConfig configForWrite = new TxnLogBufferedWriterConfig();
+        configForWrite.setBatchEnabled(writeWithBatch);
+        configForWrite.setBatchedWriteMaxRecords(2);
+        // Denied scheduled flush.
+        configForWrite.setBatchedWriteMaxDelayInMillis(1000 * 3600);
+        MLPendingAckStore mlPendingAckStoreForWrite = createPendingAckStore(configForWrite);
+        List<CompletableFuture<Void>> futureList = new ArrayList<>();
+        for (int i = 0; i < 20; i++){
+            TxnID txnID = new TxnID(i, i);
+            PositionImpl position = PositionImpl.get(i, i);
+            futureList.add(mlPendingAckStoreForWrite.appendCumulativeAck(txnID, position));
+        }
+        for (int i = 0; i < 10; i++){
+            TxnID txnID = new TxnID(i, i);
+            futureList.add(mlPendingAckStoreForWrite.appendCommitMark(txnID, CommandAck.AckType.Cumulative));
+        }
+        for (int i = 10; i < 20; i++){
+            TxnID txnID = new TxnID(i, i);
+            futureList.add(mlPendingAckStoreForWrite.appendAbortMark(txnID, CommandAck.AckType.Cumulative));
+        }
+        for (int i = 40; i < 50; i++){
+            TxnID txnID = new TxnID(i, i);
+            PositionImpl position = PositionImpl.get(i, i);
+            futureList.add(mlPendingAckStoreForWrite.appendCumulativeAck(txnID, position));
+        }
+        FutureUtil.waitForAll(futureList).get();
+        // Verify build sparse indexes correct after add many cmd-ack.
+        ArrayList<Long> positionList = new ArrayList<>();
+        for (long i = 0; i < 50; i++){
+            positionList.add(i);
+        }
+        // The indexes not contains the data which is commit or abort.
+        LinkedHashSet<Long> skipSet = new LinkedHashSet<>();
+        for (long i = 20; i < 40; i++){
+            skipSet.add(i);
+        }
+        if (writeWithBatch) {
+            for (long i = 0; i < 50; i++){
+                if (i % 2 == 0){
+                    // The indexes contains only the last position in the batch.
+                    skipSet.add(i);
+                }
+            }
+        }
+        LinkedHashSet<Long> expectedPositions = calculatePendingAckIndexes(positionList, skipSet);
+        Assert.assertEquals(
+                mlPendingAckStoreForWrite.pendingAckLogIndex.keySet().stream()
+                        .map(PositionImpl::getEntryId).collect(Collectors.toList()),
+                new ArrayList<>(expectedPositions)
+        );
+        // Replay.
+        TxnLogBufferedWriterConfig configForReplay = new TxnLogBufferedWriterConfig();
+        configForReplay.setBatchEnabled(readWithBatch);
+        configForReplay.setBatchedWriteMaxRecords(2);
+        // Denied scheduled flush.
+        configForReplay.setBatchedWriteMaxDelayInMillis(1000 * 3600);
+        MLPendingAckStore mlPendingAckStoreForRead = createPendingAckStore(configForReplay);
+        PendingAckHandleImpl pendingAckHandle = mock(PendingAckHandleImpl.class);
+        when(pendingAckHandle.getInternalPinnedExecutor()).thenReturn(internalPinnedExecutor);
+        when(pendingAckHandle.changeToReadyState()).thenReturn(true);
+        // Process controller, mark the replay task already finish.
+        final AtomicInteger processController = new AtomicInteger();
+        doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                processController.incrementAndGet();
+                return null;
+            }
+        }).when(pendingAckHandle).completeHandleFuture();
+        mlPendingAckStoreForRead.replayAsync(pendingAckHandle, internalPinnedExecutor);
+        Awaitility.await().atMost(200, TimeUnit.SECONDS).until(() -> processController.get() == 1);

Review Comment:
   Why need to wait at most 200 seconds? we don't have too many logs in the pending ack store.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/MLPendingAckStore.java:
##########
@@ -362,6 +439,25 @@ public void run() {
         }
     }
 
+    private List<PendingAckMetadataEntry> deserializeEntry(Entry entry){
+        ByteBuf buffer = entry.getDataBuffer();
+        // Check whether it is batched Entry.
+        buffer.markReaderIndex();
+        short magicNum = buffer.readShort();
+        buffer.resetReaderIndex();
+        if (magicNum == BATCHED_ENTRY_DATA_PREFIX_MAGIC_NUMBER){
+            // skip version
+            buffer.skipBytes(4);

Review Comment:
   Use the constant https://github.com/apache/pulsar/pull/16685/files#diff-4fb66809c520962f28616ddcab91aca0abb7aa04ee3ef0703097cf5976080ba8R301-R303



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

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

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