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/20 11:00:49 UTC

[GitHub] [pulsar] congbobo184 commented on a diff in pull request #16685: [improve][txn] PIP-160: Transaction log store enables the batch feature

congbobo184 commented on code in PR #16685:
URL: https://github.com/apache/pulsar/pull/16685#discussion_r925354353


##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java:
##########
@@ -209,15 +238,46 @@ class TransactionLogReplayer {
         }
 
         public void start() {
-            TransactionMetadataEntry transactionMetadataEntry = new TransactionMetadataEntry();
-
             while (fillEntryQueueCallback.fillQueue() || entryQueue.size() > 0) {
                 Entry entry = entryQueue.poll();
                 if (entry != null) {
                     try {
-                        ByteBuf buffer = entry.getDataBuffer();
-                        transactionMetadataEntry.parseFrom(buffer, buffer.readableBytes());
-                        transactionLogReplayCallback.handleMetadataEntry(entry.getPosition(), transactionMetadataEntry);
+                        List<TransactionMetadataEntry> logs = deserializeEntry(entry);

Review Comment:
   If check the entry is batch entry first, we don't need return list when the entry is not batch entry



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java:
##########
@@ -174,6 +197,12 @@ public void addFailed(ManagedLedgerException exception, Object ctx) {
 
     public CompletableFuture<Void> deletePosition(List<Position> positions) {
         CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        // Change the flag in ackSet to deleted.
+        for (Position position : positions) {
+            if (position instanceof TxnBatchedPositionImpl batchedPosition){
+                batchedPosition.deleteFromAckSet();

Review Comment:
   If add complete return 11011, index 2, is this add op batch index? We don't need this step, right?



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/TxnLogBufferedWriterConfig.java:
##########
@@ -0,0 +1,30 @@
+/**
+ * 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 lombok.Data;
+
+@Data
+public class TxnLogBufferedWriterConfig {
+
+    private int batchedWriteMaxRecords = 512;
+    private int batchedWriteMaxSize = 1024 * 1024 * 4;
+    private int batchedWriteMaxDelayInMillis = 1;
+    private boolean batchEnabled = false;

Review Comment:
   default enable



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java:
##########
@@ -283,4 +365,43 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
         }
 
     }
+
+    private static final FastThreadLocal<BatchedTransactionMetadataEntry> localBatchedTransactionLogCache =
+            new FastThreadLocal<>() {
+                @Override
+                protected BatchedTransactionMetadataEntry initialValue() throws Exception {
+                    return new BatchedTransactionMetadataEntry();
+                }
+            };

Review Comment:
   This doesn't seem to work because there is no recycle



##########
pulsar-transaction/coordinator/src/main/java/org/apache/pulsar/transaction/coordinator/impl/MLTransactionLogImpl.java:
##########
@@ -209,15 +238,46 @@ class TransactionLogReplayer {
         }
 
         public void start() {
-            TransactionMetadataEntry transactionMetadataEntry = new TransactionMetadataEntry();
-
             while (fillEntryQueueCallback.fillQueue() || entryQueue.size() > 0) {
                 Entry entry = entryQueue.poll();
                 if (entry != null) {
                     try {
-                        ByteBuf buffer = entry.getDataBuffer();
-                        transactionMetadataEntry.parseFrom(buffer, buffer.readableBytes());
-                        transactionLogReplayCallback.handleMetadataEntry(entry.getPosition(), transactionMetadataEntry);
+                        List<TransactionMetadataEntry> logs = deserializeEntry(entry);
+                        if (logs.isEmpty()){
+                            continue;
+                        } else if (logs.size() == 1){
+                            TransactionMetadataEntry log = logs.get(0);
+                            transactionLogReplayCallback.handleMetadataEntry(entry.getPosition(), log);
+                        } else {
+                            /**
+                             * 1. Query batch index of current entry from cursor.
+                             * 2. Filter the data which has already ack.
+                             * 3. Build batched position and handle valid data.
+                             */
+                            long[] ackSetAlreadyAck = cursor.getDeletedBatchIndexesAsLongArray(
+                                    PositionImpl.get(entry.getLedgerId(), entry.getEntryId()));
+                            BitSetRecyclable bitSetAlreadyAck = null;
+                            if (ackSetAlreadyAck != null){
+                                bitSetAlreadyAck = BitSetRecyclable.valueOf(ackSetAlreadyAck);
+                            }
+                            int batchSize = logs.size();
+                            for (int batchIndex = 0; batchIndex < batchSize; batchIndex++){
+                                if (bitSetAlreadyAck != null && !bitSetAlreadyAck.get(batchIndex)){
+                                   continue;
+                                }
+                                TransactionMetadataEntry log = logs.get(batchIndex);
+                                BitSetRecyclable bitSetOfCurrentRecord = BitSetRecyclable.create();
+                                bitSetOfCurrentRecord.set(batchIndex);
+                                long[] ackSetOfCurrentRecord = bitSetOfCurrentRecord.toLongArray();
+                                bitSetOfCurrentRecord.recycle();
+                                PositionImpl batchedPosition = PositionImpl.get(entry.getLedgerId(),
+                                        entry.getEntryId(), ackSetOfCurrentRecord);

Review Comment:
   because the position is only used to delete the log data, only for cursor ack the position, so I think set(0, batchSize - 1) and clear(batch index) si better, otherwise when you ack the position should flip the position



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

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

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