liangyepianzhou commented on code in PR #18273:
URL: https://github.com/apache/pulsar/pull/18273#discussion_r1021388094


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,599 @@
+/**
+ * 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,2
+ * 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.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import 
org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import 
org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import 
org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import 
org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+import org.apache.pulsar.common.util.collections.ConcurrentOpenHashSet;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements 
AbortedTxnProcessor {
+
+    private ConcurrentOpenHashSet<TxnID> unsealedAbortedTxnIdSegment;
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, 
ConcurrentOpenHashSet<TxnID>> abortTxnSegments =
+            new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, 
TransactionBufferSnapshotIndex> indexes =
+            new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new 
segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new 
TransactionBufferSnapshotIndexes();
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                
.getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  
topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.unsealedAbortedTxnIdSegment =
+                new 
ConcurrentOpenHashSet<>(this.transactionBufferMaxAbortedTxnsOfSnapshotSegment, 
1);
+    }
+
+    @Override
+    public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl 
abortedMarkerPersistentPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of 
snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == 
transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            ConcurrentOpenHashSet<TxnID> abortedSegment = 
unsealedAbortedTxnIdSegment;
+            abortTxnSegments.put(abortedMarkerPersistentPosition, 
abortedSegment);
+            
persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(abortedSegment, 
this.topic.getMaxReadPosition()));
+            this.unsealedAbortedTxnIdSegment =
+                    new 
ConcurrentOpenHashSet<>(this.transactionBufferMaxAbortedTxnsOfSnapshotSegment, 
1);
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) 
{
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || 
unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = 
abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.get(maxReadPosition).contains(txnID);
+            } else {
+                return unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    //In this implementation, we adopt snapshot segments. And then we clear 
invalid segment by its max read position.
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) 
topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted 
transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            
persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> 
persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnsSnapshot(PositionImpl 
maxReadPosition) {
+        TransactionBufferSnapshotIndexesMetadata metadata = new 
TransactionBufferSnapshotIndexesMetadata(
+                maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(),
+                serializationForSegment(unsealedAbortedTxnIdSegment));
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                () -> persistentWorker
+                        .updateSnapshotIndex(metadata, 
persistentSnapshotIndexes.getIndexList())
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return 
topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                
.createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, 
lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message 
= reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes 
transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = 
transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            
transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            
transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when 
read "
+                                + "transactionBufferSnapshot!", 
topic.getName(), ex);
+                        return FutureUtil.failedFuture(ex);
+                    } finally {
+                        closeReader(reader);
+                    }
+                    PositionImpl finalStartReadCursorPosition = 
startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                        
transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        
transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                
transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = 
deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        if (indexes.size() != 0) {
+                            
persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                        }
+                    }
+                    //Read snapshot segment to recover aborts.
+                    ArrayList<CompletableFuture<Void>> completableFutures = 
new ArrayList<>();
+                    CompletableFuture<Void> openManagedLedgerFuture = new 
CompletableFuture<>();
+                    AtomicLong invalidIndex = new AtomicLong(0);
+                    AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback 
= new AsyncCallbacks
+                            .OpenReadOnlyManagedLedgerCallback() {
+                        @Override
+                        public void 
openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl 
readOnlyManagedLedger,
+                                                                      Object 
ctx) {
+                            
persistentSnapshotIndexes.getIndexList().forEach(index -> {
+                                CompletableFuture<Void> handleSegmentFuture = 
new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new 
PositionImpl(index.getPersistentPositionLedgerID(),
+                                                
index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() 
{
+                                            @Override
+                                            public void 
readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new 
PositionImpl(
+                                                            
index.getMaxReadPositionLedgerID(),
+                                                            
index.getMaxReadPositionEntryID()));
+                                                    
handleSegmentFuture.complete(null);
+                                                    
invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                
handleSnapshotSegmentEntry(entry);
+                                                
handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void 
readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                
handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                            openManagedLedgerFuture.complete(null);
+                        }
+
+                        @Override
+                        public void 
openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) {
+                            log.error("[{}] Failed to open readOnly managed 
ledger", topic, exception);
+                            
openManagedLedgerFuture.completeExceptionally(exception);
+                        }
+                    };
+
+                    TopicName snapshotIndexTopicName = 
TopicName.get(TopicDomain.persistent.toString(),
+                            
TopicName.get(topic.getName()).getNamespaceObject(),
+                            
EventType.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS.toString());
+                    
this.topic.getBrokerService().getPulsar().getManagedLedgerFactory()
+                            
.asyncOpenReadOnlyManagedLedger(snapshotIndexTopicName
+                                            .getPersistenceNamingEncoding(), 
callback,
+                                    topic.getManagedLedger().getConfig(),
+                                    null);
+                    //Wait the processor recover completely and the allow TB 
to recover the messages
+                    // after the startReadCursorPosition.
+
+                    return openManagedLedgerFuture
+                            .thenCompose((ignore) -> 
FutureUtil.waitForAll(completableFutures).thenCompose((i) -> {
+                                if (invalidIndex.get() != 0) {
+                                    
persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                            -> persistentWorker
+                                            
.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                            
indexes.values().stream().toList()));
+                                }
+                                return 
CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                            })).exceptionally(ex -> {
+                                log.error("[{}] Failed to recover snapshot 
segment", this.topic.getName(), ex);
+                                return null;
+                            });
+
+                    },  
topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> deleteAbortedTxnSnapshot() {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;

Review Comment:
   It can be understood to only change in `updateSnapshotIndex`.
   When the latest max read position and aborted txn id are persistent in the 
index snapshot, lastSnapshotTimestamps will be updated.
   



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to