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/10/27 16:20:29 UTC

[GitHub] [pulsar] poorbarcode commented on a diff in pull request #17847: [feat][txn] Implement the AbortedTxnProcessor for TransactionBuffer

poorbarcode commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1006965312


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.buffer;
+
+import io.netty.util.TimerTask;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferRecoverCallBack;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+
+
+public interface AbortedTxnProcessor extends TimerTask {
+
+    /**
+     * After the transaction buffer writes a transaction aborted mark to the topic,
+     * the transaction buffer will add the aborted transaction ID to AbortedTxnProcessor.
+     * @param txnID aborted transaction ID.
+     */
+    void appendAbortedTxn(TxnIDData txnID, PositionImpl position);
+
+    /**
+     * After the transaction buffer writes a transaction aborted mark to the topic,
+     * the transaction buffer will update max read position in AbortedTxnProcessor
+     * @param maxReadPosition  the max read position after the transaction is aborted.
+     */
+    void updateMaxReadPosition(Position maxReadPosition);
+
+    /**
+     * This method is used to updated max read position for the topic which nerver used transaction send message.
+     * @param maxReadPosition the max read position after the transaction is aborted.
+     */
+    void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition);
+
+    /**
+     * Pulsar has a configuration for ledger retention time.
+     * If the transaction aborted mark position has been deleted, the transaction is valid and can be clear.
+     * In the old implementation we clear the invalid aborted txn ID one by one.
+     * In the new implementation, we adopt snapshot segments. And then we clear invalid segment by its max read position.
+     */
+    void trimExpiredTxnIDDataOrSnapshotSegments();
+
+    /**
+     * Check whether the transaction ID is an aborted transaction ID.
+     * @param txnID the transaction ID that needs to be checked.
+     * @param readPosition the read position of the transaction message, can be used to find the segment.
+     * @return a boolean, whether the transaction ID is an aborted transaction ID.
+     */
+    boolean checkAbortedTransaction(TxnIDData  txnID, Position readPosition);
+
+    /**
+     * Recover transaction buffer by transaction buffer snapshot.
+     * @return a pair consists of a Boolean if the transaction buffer needs to recover and a Position (startReadCursorPosition) determiner where to start to recover in the original topic.
+     */
+
+    CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack);

Review Comment:
   Now this method has both `@param callback` and `@return CompletableFuture<PositionImpl>`, I think we need to avoid such a complicated method-design



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -646,82 +560,57 @@ public void run() {
                             this, topic.getName());
                     return;
                 }
-                topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
-                        .getTxnBufferSnapshotService().createReader(TopicName.get(topic.getName()))
-                        .thenAcceptAsync(reader -> {
-                            try {
-                                boolean hasSnapshot = false;
-                                while (reader.hasMoreEvents()) {
-                                    Message<TransactionBufferSnapshot> message = reader.readNext();
-                                    if (topic.getName().equals(message.getKey())) {
-                                        TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
-                                        if (transactionBufferSnapshot != null) {
-                                            hasSnapshot = true;
-                                            callBack.handleSnapshot(transactionBufferSnapshot);
-                                            this.startReadCursorPosition = PositionImpl.get(
-                                                    transactionBufferSnapshot.getMaxReadPositionLedgerId(),
-                                                    transactionBufferSnapshot.getMaxReadPositionEntryId());
-                                        }
-                                    }
-                                }
-                                if (!hasSnapshot) {
-                                    closeReader(reader);
-                                    callBack.noNeedToRecover();
-                                    return;
+                abortedTxnProcessor.recoverFromSnapshot(callBack).thenAcceptAsync(startReadCursorPosition -> {

Review Comment:
   If we write the code this way, the `AbortedTxnProcessor` won't be able to care about the `callback`, which will reduce complexity: only `TopicTransactionBufferRecoverCallBack` care about `TopicTransactionBufferRecover`
   
   ```java
   final callback;
   abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> {
     if (startReadCursorPosition == null){
         callBack.noNeedToRecover();
     }else {
       ...
     }
   }).exceptionly(ex -> callBack.recoverExceptionally(ex))
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,646 @@
+/**
+ * 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.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+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.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.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;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnIDData> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnIDData>> 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 Timer timer;
+
+    private final PersistentTopic topic;
+
+    //When add abort or change max read position, the count will +1. Take snapshot will set 0 into it.
+    private final AtomicLong changeMaxReadPositionAndAddAbortTimes = new AtomicLong();
+
+    private volatile long lastSnapshotTimestamps;
+
+    //Configurations
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+
+    //Persistent snapshot segment and index at the single thread.
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+            snapshotSegmentsWriterFuture;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+            snapshotIndexWriterFuture;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+        snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnIDData abortedTxnId, PositionImpl position) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            abortTxnSegments.put(position, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, position));
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position position) {
+        if (position != this.maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) position;
+            updateSnapshotIndexMetadataByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnIDData txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        }
+        Map.Entry<PositionImpl, ArrayList<TxnIDData>> ceilingEntry = abortTxnSegments
+                .ceilingEntry((PositionImpl) readPosition);
+        if (ceilingEntry == null) {
+            return unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            return ceilingEntry.getValue().contains(txnID);
+        }
+    }
+
+    @Override
+    public void trimExpiredTxnIDDataOrSnapshotSegments() {
+        //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();
+    }
+
+    private void updateSnapshotIndexMetadataByChangeTimes() {
+        if (this.changeMaxReadPositionAndAddAbortTimes.incrementAndGet() == takeSnapshotIntervalNumber) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                    persistentWorker::updateIndexMetadataForTheLastSnapshot);
+        }
+    }
+
+    private void takeSnapshotByTimeout() {
+        if (changeMaxReadPositionAndAddAbortTimes.get() > 0) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                    persistentWorker::updateIndexMetadataForTheLastSnapshot);
+        }
+        timer.newTimeout(SnapshotSegmentAbortedTxnProcessorImpl.this,
+                takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void run(Timeout timeout) {
+        takeSnapshotByTimeout();
+    }
+
+
+    @Override
+    public CompletableFuture<Void> takesFirstSnapshot() {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, () -> {
+            TransactionBufferSnapshotIndexes indexes = new TransactionBufferSnapshotIndexes();
+            return snapshotIndexWriterFuture
+                    .thenCompose((indexesWriter) -> {
+                        TransactionBufferSnapshotIndexesMetadata transactionBufferSnapshotIndexesMetadata =
+                                new TransactionBufferSnapshotIndexesMetadata();
+                        transactionBufferSnapshotIndexesMetadata.setAborts(unsealedAbortedTxnIdSegment);
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+                        indexes.setSnapshot(transactionBufferSnapshotIndexesMetadata);
+                        indexes.setIndexList(new ArrayList<>());
+                        indexes.setTopicName(this.topic.getName());
+                        return indexesWriter.writeAsync(topic.getName(), indexes);
+                    })
+                    .thenRun(() -> {
+                        persistentSnapshotIndexes.setSnapshot(indexes.getSnapshot());
+                        persistentSnapshotIndexes.setIndexList(indexes.getIndexList());
+                        persistentSnapshotIndexes.setTopicName(this.topic.getName());
+                        this.lastSnapshotTimestamps = System.currentTimeMillis();
+                        completableFuture.complete(null);
+                    })
+                    .exceptionally(e -> {
+                        log.error("[{}] Failed to update snapshot segment index", indexes.getTopicName(), e);
+                        completableFuture.completeExceptionally(e);
+                        return null;
+                    });
+        });
+        return completableFuture;
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack) {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        boolean hasIndex = false;
+                        //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());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasIndex) {
+                            callBack.noNeedToRecover();
+                            return CompletableFuture.completedFuture(null);
+                        } else {
+                            persistentSnapshotIndexes.getIndexList()
+                                    .forEach(transactionBufferSnapshotIndex ->
+                                            indexes.put(new PositionImpl(
+                                                    transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                            transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                    transactionBufferSnapshotIndex));
+                            this.unsealedAbortedTxnIdSegment = (ArrayList<TxnIDData>) persistentSnapshotIndexes
+                                    .getSnapshot().getAborts();
+                            this.maxReadPosition = new PositionImpl(persistentSnapshotIndexes
+                                    .getSnapshot().getMaxReadPositionLedgerId(),
+                                    persistentSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                            if (indexes.size() != 0) {
+                                persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                            }
+                        }
+                        //Read snapshot segment to recover aborts.
+                        ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+                        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> completableFuture1 = new CompletableFuture<>();

Review Comment:
   `completableFuture1 ` Recommended meaningful variable names.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,563 @@
+/**
+ * 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.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+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.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.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;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    public enum OperationState {
+        None,
+        UpdatingIndex,
+        TakingSegment,
+        DeletingSegment,
+        Closing,
+        Closed
+    }
+
+    private volatile OperationState operationState = OperationState.None;
+    private final AtomicLong sequenceID = new AtomicLong(0);
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnIDData> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnIDData>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    //The queue of snapshot segment, the writer of snapshot segment will write segment in the order of the queue.
+    //Only one segment can be written at the same time.
+    //Remove the segment from the queue only when persistent successfully.
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnIDData>> snapshotSegmentQueue
+            = new ConcurrentSkipListMap<>();
+
+    private static final AtomicReferenceFieldUpdater<SnapshotSegmentAbortedTxnProcessorImpl,
+            SnapshotSegmentAbortedTxnProcessorImpl.OperationState> STATE_UPDATER =
+            AtomicReferenceFieldUpdater.newUpdater(SnapshotSegmentAbortedTxnProcessorImpl.class,
+                    SnapshotSegmentAbortedTxnProcessorImpl.OperationState.class, "operationState");
+
+    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 Timer timer;
+
+    private final PersistentTopic topic;
+
+    //When add abort or change max read position, the count will +1. Take snapshot will set 0 into it.
+    private final AtomicLong changeMaxReadPositionAndAddAbortTimes = new AtomicLong();
+
+    private volatile long lastSnapshotTimestamps;
+
+    //Configurations
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+
+    //Persistent snapshot segment and index at the single thread.
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+            snapshotSegmentsWriterFuture;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+            snapshotIndexWriterFuture;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+        snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnIDData abortedTxnId, PositionImpl position) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            abortTxnSegments.put(position, unsealedAbortedTxnIdSegment);
+            //Guarantee the order of the segments.
+            snapshotSegmentQueue.put(position, unsealedAbortedTxnIdSegment);
+            takeSnapshotSegment();
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    private void takeSnapshotSegment() {
+        //Only one segment can be written at the same time.
+        if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.TakingSegment)) {
+            ArrayList<TxnIDData> abortedTxns = snapshotSegmentQueue.firstEntry().getValue();
+            PositionImpl maxReadPosition = snapshotSegmentQueue.firstKey();
+
+            takeSnapshotSegmentAsync(abortedTxns, maxReadPosition).thenRun(() -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Successes to take snapshot segment [{}] at maxReadPosition [{}] "
+                                    + "for the topic [{}], and the size of the segment is [{}]",
+                            sequenceID, maxReadPosition, topic.getName(), abortedTxns.size());
+                }
+                snapshotSegmentQueue.remove(maxReadPosition);
+                sequenceID.getAndIncrement();
+            }).exceptionally(e -> {
+                //Just log the error, and the processor will try to take snapshot again when the transactionBuffer
+                //append aborted txn nex time.
+                log.error("Failed to take snapshot segment [{}] at maxReadPosition [{}] "
+                                + "for the topic [{}], and the size of the segment is [{}]",
+                        sequenceID, maxReadPosition, topic.getName(), abortedTxns.size(), e);
+                //Try again
+                timer.newTimeout((ignore) -> takeSnapshotSegment(), takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                return null;
+            });
+        }
+    }
+
+
+    @Override
+    public void updateMaxReadPosition(Position position) {
+        if (position != this.maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) position;
+            updateSnapshotIndexMetadataByChangeTimes();
+        }
+    }
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+
+    @Override
+    public boolean checkAbortedTransaction(TxnIDData txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        }
+        Map.Entry<PositionImpl, ArrayList<TxnIDData>> ceilingEntry = abortTxnSegments
+                .ceilingEntry((PositionImpl) readPosition);
+        if (ceilingEntry == null) {
+            return unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            return ceilingEntry.getValue().contains(txnID);
+        }
+    }
+
+    @Override
+    public void trimExpiredTxnIDDataOrSnapshotSegments() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())
+                && STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.DeletingSegment)) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            deleteSnapshotSegment(positionNeedToDelete);
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    private void updateSnapshotIndexMetadataByChangeTimes() {
+        if (this.changeMaxReadPositionAndAddAbortTimes.incrementAndGet() == takeSnapshotIntervalNumber) {
+            if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                changeMaxReadPositionAndAddAbortTimes.set(0);
+                if (snapshotSegmentQueue.isEmpty()) {
+                    updateIndexMetadataForTheLastSnapshot();
+                } else {
+                    takeSnapshotSegment();
+                }
+            }
+        }
+    }
+
+    private void takeSnapshotByTimeout() {
+        if (changeMaxReadPositionAndAddAbortTimes.get() > 0) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                if (snapshotSegmentQueue.isEmpty()) {
+                    updateIndexMetadataForTheLastSnapshot();
+                } else {
+                    takeSnapshotSegment();
+                }
+            }
+        }
+        timer.newTimeout(SnapshotSegmentAbortedTxnProcessorImpl.this,
+                takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void run(Timeout timeout) {
+        takeSnapshotByTimeout();
+    }
+
+    private CompletableFuture<Void> takeSnapshotSegmentAsync(List<TxnIDData> segment, PositionImpl maxReadPosition) {
+        TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment();
+        transactionBufferSnapshotSegment.setAborts(segment);
+        transactionBufferSnapshotSegment.setTopicName(this.topic.getName());
+        transactionBufferSnapshotSegment.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+        transactionBufferSnapshotSegment.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+
+        return snapshotSegmentsWriterFuture.thenCompose(segmentWriter -> {
+            transactionBufferSnapshotSegment.setSequenceId(this.sequenceID.get());
+            return segmentWriter.writeAsync(buildKey(this.sequenceID.get()), transactionBufferSnapshotSegment);
+        }).thenCompose((messageId) -> {
+            //Build index for this segment
+            TransactionBufferSnapshotIndex index = new TransactionBufferSnapshotIndex();
+            index.setSequenceID(transactionBufferSnapshotSegment.getSequenceId());
+            index.setMaxReadPositionLedgerID(maxReadPosition.getLedgerId());
+            index.setMaxReadPositionEntryID(maxReadPosition.getEntryId());
+            index.setPersistentPositionLedgerID(((MessageIdImpl) messageId).getLedgerId());
+            index.setPersistentPositionEntryID(((MessageIdImpl) messageId).getEntryId());
+
+            indexes.put(maxReadPosition, index);
+            //update snapshot segment index.
+            return updateSnapshotIndex(maxReadPosition, new ArrayList<>());
+        });
+    }
+
+    //Update the indexes and metadata in the transactionBufferSnapshotIndexe.
+    //Concurrency control is performed by snapshotIndexWriterFuture.
+    private CompletableFuture<Void> updateSnapshotIndex(PositionImpl maxReadPosition,
+                                                        ArrayList<TxnIDData> unsealedAbortedTxnIdSegment) {
+        TransactionBufferSnapshotIndexes snapshotIndexes = new TransactionBufferSnapshotIndexes();
+        return snapshotIndexWriterFuture
+                .thenCompose((indexesWriter) -> {
+                    snapshotIndexes.setIndexList(indexes.values().stream().toList());
+                    //update the metadata in the idnexes.
+                    snapshotIndexes.setSnapshot(new TransactionBufferSnapshotIndexesMetadata(
+                            maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), unsealedAbortedTxnIdSegment));
+                    return indexesWriter.writeAsync(topic.getName(), snapshotIndexes);
+                })
+                .thenRun(() -> {
+                    persistentSnapshotIndexes.setIndexList(snapshotIndexes.getIndexList());
+                    this.lastSnapshotTimestamps = System.currentTimeMillis();
+                    STATE_UPDATER.set(this, OperationState.None);
+                })
+                .exceptionally(e -> {
+                    log.error("[{}] Failed to update snapshot segment index", snapshotIndexes.getTopicName(), e);
+                    STATE_UPDATER.set(this, OperationState.None);
+                    return null;
+                });
+    }
+
+    //Only update the metadata in the transactionBufferSnapshotIndexes.
+    //Concurrency control is performed by snapshotIndexWriterFuture.
+    private void updateIndexMetadataForTheLastSnapshot() {
+        TransactionBufferSnapshotIndexes indexes = new TransactionBufferSnapshotIndexes();
+        snapshotIndexWriterFuture
+            .thenCompose((indexesWriter) -> {
+                //Store the latest metadata
+                TransactionBufferSnapshotIndexesMetadata transactionBufferSnapshotSegment =
+                        new TransactionBufferSnapshotIndexesMetadata();
+                transactionBufferSnapshotSegment.setAborts(unsealedAbortedTxnIdSegment);
+                indexes.setSnapshot(transactionBufferSnapshotSegment);
+                //Only update the metadata in indexes and keep the index in indexes unchanged.
+                indexes.setIndexList(persistentSnapshotIndexes.getIndexList());
+                return indexesWriter.writeAsync(topic.getName(), indexes);
+            })
+            .thenRun(() -> {
+                persistentSnapshotIndexes.setSnapshot(indexes.getSnapshot());
+                STATE_UPDATER.set(this, OperationState.None);
+                this.lastSnapshotTimestamps = System.currentTimeMillis();
+            })
+            .exceptionally(e -> {
+                STATE_UPDATER.set(this, OperationState.None);
+                log.error("[{}] Failed to update snapshot segment index", indexes.getTopicName(), e);
+                return null;
+            });
+
+    }
+
+    @Override
+    public CompletableFuture<Void> takesFirstSnapshot() {
+        if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+            TransactionBufferSnapshotIndexes indexes = new TransactionBufferSnapshotIndexes();
+            return snapshotIndexWriterFuture
+                    .thenCompose((indexesWriter) -> {
+                        TransactionBufferSnapshotIndexesMetadata transactionBufferSnapshotIndexesMetadata =
+                                new TransactionBufferSnapshotIndexesMetadata();
+                        transactionBufferSnapshotIndexesMetadata.setAborts(unsealedAbortedTxnIdSegment);
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+                        indexes.setSnapshot(transactionBufferSnapshotIndexesMetadata);
+                        indexes.setIndexList(new ArrayList<>());
+                        indexes.setTopicName(this.topic.getName());
+                        return indexesWriter.writeAsync(topic.getName(), indexes);
+                    })
+                    .thenRun(() -> {
+                        //TODO: check again
+                        persistentSnapshotIndexes.setSnapshot(indexes.getSnapshot());
+                        indexes.setIndexList(new ArrayList<>());
+                        indexes.setTopicName(this.topic.getName());
+                        this.lastSnapshotTimestamps = System.currentTimeMillis();
+                    })
+                    .exceptionally(e -> {
+                        log.error("[{}] Failed to update snapshot segment index", indexes.getTopicName(), e);
+                        return null;
+                    });
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack) {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        boolean hasIndex = false;
+                        //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();

Review Comment:
   Should we just read the last entry here?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.buffer;
+
+import io.netty.util.TimerTask;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferRecoverCallBack;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+
+
+public interface AbortedTxnProcessor extends TimerTask {
+
+    /**
+     * After the transaction buffer writes a transaction aborted mark to the topic,
+     * the transaction buffer will add the aborted transaction ID to AbortedTxnProcessor.
+     * @param txnID aborted transaction ID.
+     */
+    void appendAbortedTxn(TxnIDData txnID, PositionImpl position);
+
+    /**
+     * After the transaction buffer writes a transaction aborted mark to the topic,
+     * the transaction buffer will update max read position in AbortedTxnProcessor
+     * @param maxReadPosition  the max read position after the transaction is aborted.
+     */
+    void updateMaxReadPosition(Position maxReadPosition);
+
+    /**
+     * This method is used to updated max read position for the topic which nerver used transaction send message.
+     * @param maxReadPosition the max read position after the transaction is aborted.
+     */
+    void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition);
+
+    /**
+     * Pulsar has a configuration for ledger retention time.
+     * If the transaction aborted mark position has been deleted, the transaction is valid and can be clear.
+     * In the old implementation we clear the invalid aborted txn ID one by one.
+     * In the new implementation, we adopt snapshot segments. And then we clear invalid segment by its max read position.
+     */
+    void trimExpiredTxnIDDataOrSnapshotSegments();
+
+    /**
+     * Check whether the transaction ID is an aborted transaction ID.
+     * @param txnID the transaction ID that needs to be checked.
+     * @param readPosition the read position of the transaction message, can be used to find the segment.
+     * @return a boolean, whether the transaction ID is an aborted transaction ID.
+     */
+    boolean checkAbortedTransaction(TxnIDData  txnID, Position readPosition);
+
+    /**
+     * Recover transaction buffer by transaction buffer snapshot.
+     * @return a pair consists of a Boolean if the transaction buffer needs to recover and a Position (startReadCursorPosition) determiner where to start to recover in the original topic.
+     */
+
+    CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack);
+
+    /**
+     * Clear the snapshot/snapshot segment and index for this topic.
+     * @return a completableFuture.
+     */
+    CompletableFuture<Void> clearSnapshot();

Review Comment:
   I see that this method is only used in scenario `topic.delete()`, so would it be better to change the method name to `clearAndCloseAsync`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -163,7 +150,10 @@ public void noNeedToRecover() {
                             // sync maxReadPosition change to LAC when TopicTransaction buffer have not recover
                             // completely the normal message have been sent to broker and state is
                             // not NoSnapshot can't sync maxReadPosition
-                            maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+                            snapshotAbortedTxnProcessor
+                                    .updateMaxReadPositionNotIncreaseChangeTimes(topic.getManagedLedger()
+                                            .getLastConfirmedEntry());

Review Comment:
   ```
   // sync maxReadPosition change to LAC when TopicTransaction buffer have not recover
   // completely the normal message have been sent to broker and state is
   // not NoSnapshot can't sync maxReadPosition
   ```
   The doc above has been outdated, right?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,563 @@
+/**
+ * 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.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+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.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.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;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    public enum OperationState {
+        None,
+        UpdatingIndex,
+        TakingSegment,
+        DeletingSegment,
+        Closing,
+        Closed
+    }
+
+    private volatile OperationState operationState = OperationState.None;
+    private final AtomicLong sequenceID = new AtomicLong(0);
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnIDData> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnIDData>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    //The queue of snapshot segment, the writer of snapshot segment will write segment in the order of the queue.
+    //Only one segment can be written at the same time.
+    //Remove the segment from the queue only when persistent successfully.
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnIDData>> snapshotSegmentQueue
+            = new ConcurrentSkipListMap<>();
+
+    private static final AtomicReferenceFieldUpdater<SnapshotSegmentAbortedTxnProcessorImpl,
+            SnapshotSegmentAbortedTxnProcessorImpl.OperationState> STATE_UPDATER =
+            AtomicReferenceFieldUpdater.newUpdater(SnapshotSegmentAbortedTxnProcessorImpl.class,
+                    SnapshotSegmentAbortedTxnProcessorImpl.OperationState.class, "operationState");
+
+    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 Timer timer;
+
+    private final PersistentTopic topic;
+
+    //When add abort or change max read position, the count will +1. Take snapshot will set 0 into it.
+    private final AtomicLong changeMaxReadPositionAndAddAbortTimes = new AtomicLong();
+
+    private volatile long lastSnapshotTimestamps;
+
+    //Configurations
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+
+    //Persistent snapshot segment and index at the single thread.
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+            snapshotSegmentsWriterFuture;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+            snapshotIndexWriterFuture;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+        snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnIDData abortedTxnId, PositionImpl position) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            abortTxnSegments.put(position, unsealedAbortedTxnIdSegment);
+            //Guarantee the order of the segments.
+            snapshotSegmentQueue.put(position, unsealedAbortedTxnIdSegment);
+            takeSnapshotSegment();
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    private void takeSnapshotSegment() {
+        //Only one segment can be written at the same time.
+        if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.TakingSegment)) {
+            ArrayList<TxnIDData> abortedTxns = snapshotSegmentQueue.firstEntry().getValue();
+            PositionImpl maxReadPosition = snapshotSegmentQueue.firstKey();
+
+            takeSnapshotSegmentAsync(abortedTxns, maxReadPosition).thenRun(() -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Successes to take snapshot segment [{}] at maxReadPosition [{}] "
+                                    + "for the topic [{}], and the size of the segment is [{}]",
+                            sequenceID, maxReadPosition, topic.getName(), abortedTxns.size());
+                }
+                snapshotSegmentQueue.remove(maxReadPosition);
+                sequenceID.getAndIncrement();
+            }).exceptionally(e -> {
+                //Just log the error, and the processor will try to take snapshot again when the transactionBuffer
+                //append aborted txn nex time.
+                log.error("Failed to take snapshot segment [{}] at maxReadPosition [{}] "
+                                + "for the topic [{}], and the size of the segment is [{}]",
+                        sequenceID, maxReadPosition, topic.getName(), abortedTxns.size(), e);
+                //Try again
+                timer.newTimeout((ignore) -> takeSnapshotSegment(), takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                return null;
+            });
+        }
+    }
+
+
+    @Override
+    public void updateMaxReadPosition(Position position) {
+        if (position != this.maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) position;
+            updateSnapshotIndexMetadataByChangeTimes();
+        }
+    }
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+
+    @Override
+    public boolean checkAbortedTransaction(TxnIDData txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        }
+        Map.Entry<PositionImpl, ArrayList<TxnIDData>> ceilingEntry = abortTxnSegments
+                .ceilingEntry((PositionImpl) readPosition);
+        if (ceilingEntry == null) {
+            return unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            return ceilingEntry.getValue().contains(txnID);
+        }
+    }
+
+    @Override
+    public void trimExpiredTxnIDDataOrSnapshotSegments() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())
+                && STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.DeletingSegment)) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            deleteSnapshotSegment(positionNeedToDelete);
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    private void updateSnapshotIndexMetadataByChangeTimes() {
+        if (this.changeMaxReadPositionAndAddAbortTimes.incrementAndGet() == takeSnapshotIntervalNumber) {
+            if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                changeMaxReadPositionAndAddAbortTimes.set(0);
+                if (snapshotSegmentQueue.isEmpty()) {
+                    updateIndexMetadataForTheLastSnapshot();
+                } else {
+                    takeSnapshotSegment();
+                }
+            }
+        }
+    }
+
+    private void takeSnapshotByTimeout() {
+        if (changeMaxReadPositionAndAddAbortTimes.get() > 0) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                if (snapshotSegmentQueue.isEmpty()) {
+                    updateIndexMetadataForTheLastSnapshot();
+                } else {
+                    takeSnapshotSegment();
+                }
+            }
+        }
+        timer.newTimeout(SnapshotSegmentAbortedTxnProcessorImpl.this,
+                takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void run(Timeout timeout) {
+        takeSnapshotByTimeout();
+    }
+
+    private CompletableFuture<Void> takeSnapshotSegmentAsync(List<TxnIDData> segment, PositionImpl maxReadPosition) {
+        TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment();
+        transactionBufferSnapshotSegment.setAborts(segment);
+        transactionBufferSnapshotSegment.setTopicName(this.topic.getName());
+        transactionBufferSnapshotSegment.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+        transactionBufferSnapshotSegment.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+
+        return snapshotSegmentsWriterFuture.thenCompose(segmentWriter -> {
+            transactionBufferSnapshotSegment.setSequenceId(this.sequenceID.get());
+            return segmentWriter.writeAsync(buildKey(this.sequenceID.get()), transactionBufferSnapshotSegment);
+        }).thenCompose((messageId) -> {
+            //Build index for this segment
+            TransactionBufferSnapshotIndex index = new TransactionBufferSnapshotIndex();
+            index.setSequenceID(transactionBufferSnapshotSegment.getSequenceId());
+            index.setMaxReadPositionLedgerID(maxReadPosition.getLedgerId());
+            index.setMaxReadPositionEntryID(maxReadPosition.getEntryId());
+            index.setPersistentPositionLedgerID(((MessageIdImpl) messageId).getLedgerId());
+            index.setPersistentPositionEntryID(((MessageIdImpl) messageId).getEntryId());
+
+            indexes.put(maxReadPosition, index);
+            //update snapshot segment index.
+            return updateSnapshotIndex(maxReadPosition, new ArrayList<>());
+        });
+    }
+
+    //Update the indexes and metadata in the transactionBufferSnapshotIndexe.
+    //Concurrency control is performed by snapshotIndexWriterFuture.
+    private CompletableFuture<Void> updateSnapshotIndex(PositionImpl maxReadPosition,
+                                                        ArrayList<TxnIDData> unsealedAbortedTxnIdSegment) {
+        TransactionBufferSnapshotIndexes snapshotIndexes = new TransactionBufferSnapshotIndexes();
+        return snapshotIndexWriterFuture
+                .thenCompose((indexesWriter) -> {
+                    snapshotIndexes.setIndexList(indexes.values().stream().toList());
+                    //update the metadata in the idnexes.
+                    snapshotIndexes.setSnapshot(new TransactionBufferSnapshotIndexesMetadata(
+                            maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), unsealedAbortedTxnIdSegment));
+                    return indexesWriter.writeAsync(topic.getName(), snapshotIndexes);
+                })
+                .thenRun(() -> {
+                    persistentSnapshotIndexes.setIndexList(snapshotIndexes.getIndexList());
+                    this.lastSnapshotTimestamps = System.currentTimeMillis();
+                    STATE_UPDATER.set(this, OperationState.None);
+                })
+                .exceptionally(e -> {
+                    log.error("[{}] Failed to update snapshot segment index", snapshotIndexes.getTopicName(), e);
+                    STATE_UPDATER.set(this, OperationState.None);
+                    return null;
+                });
+    }
+
+    //Only update the metadata in the transactionBufferSnapshotIndexes.
+    //Concurrency control is performed by snapshotIndexWriterFuture.
+    private void updateIndexMetadataForTheLastSnapshot() {
+        TransactionBufferSnapshotIndexes indexes = new TransactionBufferSnapshotIndexes();
+        snapshotIndexWriterFuture
+            .thenCompose((indexesWriter) -> {
+                //Store the latest metadata
+                TransactionBufferSnapshotIndexesMetadata transactionBufferSnapshotSegment =
+                        new TransactionBufferSnapshotIndexesMetadata();
+                transactionBufferSnapshotSegment.setAborts(unsealedAbortedTxnIdSegment);
+                indexes.setSnapshot(transactionBufferSnapshotSegment);
+                //Only update the metadata in indexes and keep the index in indexes unchanged.
+                indexes.setIndexList(persistentSnapshotIndexes.getIndexList());
+                return indexesWriter.writeAsync(topic.getName(), indexes);
+            })
+            .thenRun(() -> {
+                persistentSnapshotIndexes.setSnapshot(indexes.getSnapshot());
+                STATE_UPDATER.set(this, OperationState.None);
+                this.lastSnapshotTimestamps = System.currentTimeMillis();
+            })
+            .exceptionally(e -> {
+                STATE_UPDATER.set(this, OperationState.None);
+                log.error("[{}] Failed to update snapshot segment index", indexes.getTopicName(), e);
+                return null;
+            });
+
+    }
+
+    @Override
+    public CompletableFuture<Void> takesFirstSnapshot() {
+        if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+            TransactionBufferSnapshotIndexes indexes = new TransactionBufferSnapshotIndexes();
+            return snapshotIndexWriterFuture
+                    .thenCompose((indexesWriter) -> {
+                        TransactionBufferSnapshotIndexesMetadata transactionBufferSnapshotIndexesMetadata =
+                                new TransactionBufferSnapshotIndexesMetadata();
+                        transactionBufferSnapshotIndexesMetadata.setAborts(unsealedAbortedTxnIdSegment);
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+                        indexes.setSnapshot(transactionBufferSnapshotIndexesMetadata);
+                        indexes.setIndexList(new ArrayList<>());
+                        indexes.setTopicName(this.topic.getName());
+                        return indexesWriter.writeAsync(topic.getName(), indexes);
+                    })
+                    .thenRun(() -> {
+                        //TODO: check again
+                        persistentSnapshotIndexes.setSnapshot(indexes.getSnapshot());
+                        indexes.setIndexList(new ArrayList<>());
+                        indexes.setTopicName(this.topic.getName());
+                        this.lastSnapshotTimestamps = System.currentTimeMillis();
+                    })
+                    .exceptionally(e -> {
+                        log.error("[{}] Failed to update snapshot segment index", indexes.getTopicName(), e);
+                        return null;
+                    });
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack) {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        boolean hasIndex = false;
+                        //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());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasIndex) {
+                            callBack.noNeedToRecover();
+                            return CompletableFuture.completedFuture(null);
+                        } else {
+                            persistentSnapshotIndexes.getIndexList()
+                                    .forEach(transactionBufferSnapshotIndex ->
+                                            indexes.put(new PositionImpl(
+                                                    transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                            transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                    transactionBufferSnapshotIndex));
+                            this.unsealedAbortedTxnIdSegment = (ArrayList<TxnIDData>) persistentSnapshotIndexes
+                                    .getSnapshot().getAborts();
+                            this.maxReadPosition = new PositionImpl(persistentSnapshotIndexes
+                                    .getSnapshot().getMaxReadPositionLedgerId(),
+                                    persistentSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                            if (indexes.size() != 0) {
+                                sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                            }
+                        }
+                        //Read snapshot segment to recover aborts.
+                        ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+                        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> completableFuture1 = new CompletableFuture<>();
+                                    completableFutures.add(completableFuture1);
+                                    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()));
+                                                        completableFuture1.complete(null);
+                                                        invalidIndex.getAndIncrement();
+                                                        return;
+                                                    }
+                                                    handleSnapshotSegmentEntry(entry);
+                                                    completableFuture1.complete(null);
+                                                }
+
+                                                @Override
+                                                public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                    completableFuture1.completeExceptionally(exception);
+                                                }
+                                            }, null);
+                                });
+                            }
+
+                            @Override
+                            public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) {
+                                //
+                            }
+                        };
+
+                        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.
+                        FutureUtil.waitForAll(completableFutures).get();
+                        if (invalidIndex.get() != 0 && STATE_UPDATER.compareAndSet(this,
+                                OperationState.None, OperationState.UpdatingIndex)) {
+                            updateSnapshotIndex(this.maxReadPosition, this.unsealedAbortedTxnIdSegment);
+                        }
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        callBack.recoverExceptionally(ex);
+                        closeReader(reader);
+                        return null;
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearSnapshot() {

Review Comment:
   Should we stop the `timer task` in this method?
   
   Same for `SingleSnapshotAbortedTxnProcessorImpl`



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,646 @@
+/**
+ * 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.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+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.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.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;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnIDData> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnIDData>> 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 Timer timer;
+
+    private final PersistentTopic topic;
+
+    //When add abort or change max read position, the count will +1. Take snapshot will set 0 into it.
+    private final AtomicLong changeMaxReadPositionAndAddAbortTimes = new AtomicLong();
+
+    private volatile long lastSnapshotTimestamps;
+
+    //Configurations
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+
+    //Persistent snapshot segment and index at the single thread.
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+            snapshotSegmentsWriterFuture;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+            snapshotIndexWriterFuture;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+        snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnIDData abortedTxnId, PositionImpl position) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            abortTxnSegments.put(position, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, position));
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position position) {
+        if (position != this.maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) position;
+            updateSnapshotIndexMetadataByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnIDData txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        }
+        Map.Entry<PositionImpl, ArrayList<TxnIDData>> ceilingEntry = abortTxnSegments
+                .ceilingEntry((PositionImpl) readPosition);
+        if (ceilingEntry == null) {
+            return unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            return ceilingEntry.getValue().contains(txnID);
+        }
+    }
+
+    @Override
+    public void trimExpiredTxnIDDataOrSnapshotSegments() {
+        //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();
+    }
+
+    private void updateSnapshotIndexMetadataByChangeTimes() {
+        if (this.changeMaxReadPositionAndAddAbortTimes.incrementAndGet() == takeSnapshotIntervalNumber) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                    persistentWorker::updateIndexMetadataForTheLastSnapshot);
+        }
+    }
+
+    private void takeSnapshotByTimeout() {
+        if (changeMaxReadPositionAndAddAbortTimes.get() > 0) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                    persistentWorker::updateIndexMetadataForTheLastSnapshot);
+        }
+        timer.newTimeout(SnapshotSegmentAbortedTxnProcessorImpl.this,
+                takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void run(Timeout timeout) {
+        takeSnapshotByTimeout();
+    }
+
+
+    @Override
+    public CompletableFuture<Void> takesFirstSnapshot() {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, () -> {
+            TransactionBufferSnapshotIndexes indexes = new TransactionBufferSnapshotIndexes();
+            return snapshotIndexWriterFuture
+                    .thenCompose((indexesWriter) -> {
+                        TransactionBufferSnapshotIndexesMetadata transactionBufferSnapshotIndexesMetadata =
+                                new TransactionBufferSnapshotIndexesMetadata();
+                        transactionBufferSnapshotIndexesMetadata.setAborts(unsealedAbortedTxnIdSegment);
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+                        transactionBufferSnapshotIndexesMetadata.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+                        indexes.setSnapshot(transactionBufferSnapshotIndexesMetadata);
+                        indexes.setIndexList(new ArrayList<>());
+                        indexes.setTopicName(this.topic.getName());
+                        return indexesWriter.writeAsync(topic.getName(), indexes);
+                    })
+                    .thenRun(() -> {
+                        persistentSnapshotIndexes.setSnapshot(indexes.getSnapshot());
+                        persistentSnapshotIndexes.setIndexList(indexes.getIndexList());
+                        persistentSnapshotIndexes.setTopicName(this.topic.getName());
+                        this.lastSnapshotTimestamps = System.currentTimeMillis();
+                        completableFuture.complete(null);
+                    })
+                    .exceptionally(e -> {
+                        log.error("[{}] Failed to update snapshot segment index", indexes.getTopicName(), e);
+                        completableFuture.completeExceptionally(e);
+                        return null;
+                    });
+        });
+        return completableFuture;
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack) {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        boolean hasIndex = false;
+                        //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());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasIndex) {
+                            callBack.noNeedToRecover();
+                            return CompletableFuture.completedFuture(null);
+                        } else {
+                            persistentSnapshotIndexes.getIndexList()
+                                    .forEach(transactionBufferSnapshotIndex ->
+                                            indexes.put(new PositionImpl(
+                                                    transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                            transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                    transactionBufferSnapshotIndex));
+                            this.unsealedAbortedTxnIdSegment = (ArrayList<TxnIDData>) persistentSnapshotIndexes
+                                    .getSnapshot().getAborts();
+                            this.maxReadPosition = new PositionImpl(persistentSnapshotIndexes
+                                    .getSnapshot().getMaxReadPositionLedgerId(),
+                                    persistentSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                            if (indexes.size() != 0) {
+                                persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                            }
+                        }
+                        //Read snapshot segment to recover aborts.
+                        ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+                        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> completableFuture1 = new CompletableFuture<>();
+                                    completableFutures.add(completableFuture1);
+                                    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()));
+                                                        completableFuture1.complete(null);
+                                                        invalidIndex.getAndIncrement();
+                                                        return;
+                                                    }
+                                                    handleSnapshotSegmentEntry(entry);

Review Comment:
   Here all `snapshot-segment` are loaded into memory, our original design should be loaded on demand. Should we left a `TODO` for later optimization?



-- 
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