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/09/27 03:55:57 UTC

[GitHub] [pulsar] liangyepianzhou opened a new pull request, #17847: [improve][txn] Implement the AbortedTxnProcessor for TransactionBuffer

liangyepianzhou opened a new pull request, #17847:
URL: https://github.com/apache/pulsar/pull/17847

   Master Issue: https://github.com/apache/pulsar/issues/16913
   
   ### Motivation
   
   Implement a abortedTxnProcessor to handle the storage of aborted transaction ID.
   ### Modifications
   ### Documentation
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   - [x] `doc-not-needed` 
   (Please explain why)
   
   
   ### Matching PR in forked repository
   
   PR in forked repository: https://github.com/liangyepianzhou/pulsar/pull/7


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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1006532810


##########
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 -> {
+                    //Transaction is not enable for this topic, so just make maxReadPosition as LAC.
+                    if (startReadCursorPosition == null) {
+                        return;
+                    } else {
+                        this.startReadCursorPosition = startReadCursorPosition;
+                    }

Review Comment:
   startReadCursorPosition = null, we don't need to recover by the orignal topic
   ```suggestion
                       if (startReadCursorPosition == null) {
                           callBack.noNeedToRecover();
                           return;
                       } else {
                           this.startReadCursorPosition = startReadCursorPosition;
                       }
   ```
   callBack don't become the parameter for the mthod `recoverFromSnapshot `



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -197,7 +173,8 @@ public void handleTxnEntry(Entry entry) {
                             PositionImpl position = PositionImpl.get(entry.getLedgerId(), entry.getEntryId());
                             if (Markers.isTxnMarker(msgMetadata)) {
                                 if (Markers.isTxnAbortMarker(msgMetadata)) {
-                                    aborts.put(txnID, position);
+                                    snapshotAbortedTxnProcessor.appendAbortedTxn(
+                                            new TxnIDData(txnID.getMostSigBits(), txnID.getLeastSigBits()), position);

Review Comment:
   ```suggestion
                                       snapshotAbortedTxnProcessor.appendAbortedTxn(txnId, position);
   ```
   TxnIDData only take snapshot need to cover



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -225,7 +202,8 @@ public void recoverExceptionally(Throwable e) {
                         recoverTime.setRecoverEndTime(System.currentTimeMillis());
                         topic.close(true);
                     }
-                }, this.topic, this, takeSnapshotWriter));
+                }, this.topic,
+                        this, takeSnapshotWriter, snapshotAbortedTxnProcessor));

Review Comment:
   ```suggestion
                   }, this.topic, this, takeSnapshotWriter, snapshotAbortedTxnProcessor));
   ```
   the takeSnapshotWriter can become a variable in snapshotAbortedTxnProcessor



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,250 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.common.naming.TopicName;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnIDData, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnIDData abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position maxReadPosition) {
+        if (this.maxReadPosition != maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) maxReadPosition;
+            takeSnapshotByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public void trimExpiredTxnIDDataOrSnapshotSegments() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                aborts.firstKey();
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnIDData txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot(TopicTransactionBufferRecoverCallBack callBack) {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    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;
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasSnapshot) {
+                            callBack.noNeedToRecover();
+                            return CompletableFuture.completedFuture(null);
+                        }
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        callBack.recoverExceptionally(ex);
+                        closeReader(reader);

Review Comment:
   future isn't completely



##########
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 -> {
+                    //Transaction is not enable for this topic, so just make maxReadPosition as LAC.
+                    if (startReadCursorPosition == null) {
+                        return;
+                    } else {
+                        this.startReadCursorPosition = startReadCursorPosition;
+                    }
+                    ManagedCursor managedCursor;
+                    try {
+                        managedCursor = topic.getManagedLedger()
+                                .newNonDurableCursor(this.startReadCursorPosition, SUBSCRIPTION_NAME);
+                    } catch (ManagedLedgerException e) {
+                        callBack.recoverExceptionally(e);
+                        log.error("[{}]Transaction buffer recover fail when open cursor!", topic.getName(), e);
+                        return;
+                    }
+                    PositionImpl lastConfirmedEntry =
+                            (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+                    PositionImpl currentLoadPosition = (PositionImpl) this.startReadCursorPosition;
+                    FillEntryQueueCallback fillEntryQueueCallback = new FillEntryQueueCallback(entryQueue,
+                            managedCursor, TopicTransactionBufferRecover.this);
+                    if (lastConfirmedEntry.getEntryId() != -1) {
+                        while (lastConfirmedEntry.compareTo(currentLoadPosition) > 0
+                                && fillEntryQueueCallback.fillQueue()) {
+                            Entry entry = entryQueue.poll();
+                            if (entry != null) {
+                                try {
+                                    currentLoadPosition = PositionImpl.get(entry.getLedgerId(),
+                                            entry.getEntryId());
+                                    callBack.handleTxnEntry(entry);
+                                } finally {
+                                    entry.release();
                                 }
-                            } catch (Exception ex) {
-                                log.error("[{}] Transaction buffer recover fail when read "
-                                        + "transactionBufferSnapshot!", topic.getName(), ex);
-                                callBack.recoverExceptionally(ex);
-                                closeReader(reader);
-                                return;
-                            }
-                            closeReader(reader);
-
-                            ManagedCursor managedCursor;
-                            try {
-                                managedCursor = topic.getManagedLedger()
-                                        .newNonDurableCursor(this.startReadCursorPosition, SUBSCRIPTION_NAME);
-                            } catch (ManagedLedgerException e) {
-                                callBack.recoverExceptionally(e);
-                                log.error("[{}]Transaction buffer recover fail when open cursor!", topic.getName(), e);
-                                return;
-                            }
-                            PositionImpl lastConfirmedEntry =
-                                    (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
-                            PositionImpl currentLoadPosition = (PositionImpl) this.startReadCursorPosition;
-                            FillEntryQueueCallback fillEntryQueueCallback = new FillEntryQueueCallback(entryQueue,
-                                    managedCursor, TopicTransactionBufferRecover.this);
-                            if (lastConfirmedEntry.getEntryId() != -1) {
-                                while (lastConfirmedEntry.compareTo(currentLoadPosition) > 0
-                                        && fillEntryQueueCallback.fillQueue()) {
-                                    Entry entry = entryQueue.poll();
-                                    if (entry != null) {
-                                        try {
-                                            currentLoadPosition = PositionImpl.get(entry.getLedgerId(),
-                                                    entry.getEntryId());
-                                            callBack.handleTxnEntry(entry);
-                                        } finally {
-                                            entry.release();
-                                        }
-                                    } else {
-                                        try {
-                                            Thread.sleep(1);
-                                        } catch (InterruptedException e) {
-                                            //no-op
-                                        }
-                                    }
+                            } else {
+                                try {
+                                    Thread.sleep(1);
+                                } catch (InterruptedException e) {
+                                    //no-op
                                 }
                             }
+                        }
+                    }
 
-                            closeCursor(SUBSCRIPTION_NAME);
-                            callBack.recoverComplete();
-                        }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
-                                .getExecutor(this)).exceptionally(e -> {
-                            callBack.recoverExceptionally(e.getCause());
-                            log.error("[{}]Transaction buffer new snapshot reader fail!", topic.getName(), e);
-                            return null;
-                        });
+                    closeCursor(SUBSCRIPTION_NAME);
+                    callBack.recoverComplete();
+                }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this)).exceptionally(e -> {
+                    callBack.recoverExceptionally(e.getCause());
+                    log.error("[{}]Transaction buffer new snapshot reader fail!", topic.getName(), e);

Review Comment:
   change this log detail



##########
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();

Review Comment:
   ```suggestion
       void trimExpiredAbortedTxns();
   ```



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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008651474


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,255 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();

Review Comment:
   better move `maxReadPosition` to TopicTransactionBuffer



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


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

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008646579


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();

Review Comment:
   The format of the Key in the proposal is `multiple-{sequence}-{topic}`, but the current is `multiple-{sequence}{topic}`.
   
   By the way, why should prefix with `multiple-`, why do not we just format it as `{topic}-seq-{sequence}`?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        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 -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @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.
+                        return FutureUtil.waitForAll(completableFutures).thenCompose((ignore) -> {
+                            if (invalidIndex.get() != 0 ) {
+                                persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                        -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                        indexes.values().stream().toList()));
+                            }
+                            return CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                        });
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public PositionImpl getMaxReadPosition() {
+        return this.maxReadPosition;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return persistentWorker.closeAsync();
+    }
+
+    private void handleSnapshotSegmentEntry(Entry entry) {
+        //decode snapshot from entry
+        ByteBuf headersAndPayload = entry.getDataBuffer();
+        //skip metadata
+        Commands.parseMessageMetadata(headersAndPayload);
+        TransactionBufferSnapshotSegment snapshotSegment = Schema.AVRO(TransactionBufferSnapshotSegment.class)
+                .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer());
+        abortTxnSegments.put(new PositionImpl(snapshotSegment.getMaxReadPositionLedgerId(),
+                snapshotSegment.getMaxReadPositionEntryId()), deserializationFotSnapshotSegment(
+                snapshotSegment.getAborts()));
+
+    }
+
+    private <T> void  closeReader(SystemTopicClient.Reader<T> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private class PersistentWorker {
+        protected final AtomicLong sequenceID = new AtomicLong(0);
+
+        private final PersistentTopic topic;
+
+        //Persistent snapshot segment and index at the single thread.
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+                snapshotSegmentsWriterFuture;
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+                snapshotIndexWriterFuture;
+
+        private enum OperationState {
+            None,
+            UpdatingIndex,
+            WritingSegment,
+            DeletingSegment,
+            Closing,
+            Closed
+        }
+        private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState>
+                STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class,
+                        PersistentWorker.OperationState.class, "operationState");
+
+        public enum OperationType {
+            UpdateIndex,
+            WriteSegment,
+            DeleteSegment,
+            Close
+        }
+
+        private volatile OperationState operationState = OperationState.None;
+
+        ConcurrentLinkedDeque<Pair<OperationType, Supplier<CompletableFuture<Void>>>> taskQueue =
+                new ConcurrentLinkedDeque<>();
+        private CompletableFuture<Void> lastOperationFuture;
+        private final Timer timer;
+
+        public PersistentWorker(PersistentTopic topic) {
+            this.topic = topic;
+            this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+            this.snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+            this.snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        }
+
+        public void appendTask(OperationType operationType, Supplier<CompletableFuture<Void>> task) {
+            switch (operationType) {
+                case UpdateIndex -> {
+                    if (!taskQueue.isEmpty()) {
+                        return;
+                    } else if(STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.whenComplete((ignore, throwable) -> {
+                            if (throwable != null && log.isDebugEnabled()) {
+                                log.debug("[{}] Failed to update index snapshot", topic.getName(), throwable);
+                            }
+
+                            STATE_UPDATER.compareAndSet(this, OperationState.UpdatingIndex, OperationState.None);
+                        });
+                    }
+                }
+                case WriteSegment, DeleteSegment -> {
+                    taskQueue.add(new MutablePair<>(operationType, task));
+                    executeTask();
+                }
+                case Close -> {
+                    STATE_UPDATER.set(this, OperationState.Closing);
+                    taskQueue.clear();
+                    lastOperationFuture.thenRun(() -> {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.thenRun(() ->
+                                STATE_UPDATER.compareAndSet(this, OperationState.Closing, OperationState.Closed));
+                    });
+                }
+            }
+        }
+
+        private void executeTask() {
+            OperationType operationType = taskQueue.getFirst().getKey();
+            switch (operationType) {
+                case WriteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.WritingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.WriteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to write snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+                                STATE_UPDATER.compareAndSet(this, OperationState.WritingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+                case DeleteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.DeletingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.DeleteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to delete snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+
+                                STATE_UPDATER.compareAndSet(this, OperationState.DeletingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+            }
+        }
+
+        protected CompletableFuture<Void> takeSnapshotSegmentAsync(ArrayList<TxnID> sealedAbortedTxnIdSegment,
+                                                                 PositionImpl maxReadPosition) {
+            return writeSnapshotSegmentAsync(sealedAbortedTxnIdSegment, maxReadPosition).thenRun(() -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Successes to take snapshot segment [{}] at maxReadPosition [{}] "
+                                    + "for the topic [{}], and the size of the segment is [{}]",
+                            this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size());
+                }
+                this.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 [{}]",
+                        this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size(), e);
+                return null;
+            });
+        }
+
+        private CompletableFuture<Void> writeSnapshotSegmentAsync(List<TxnID> segment, PositionImpl maxReadPosition) {
+            TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment();
+            transactionBufferSnapshotSegment.setAborts(serializationForSegment(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(new TransactionBufferSnapshotIndexesMetadata(
+                        maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), new ArrayList<>()),
+                        indexes.values().stream().toList());
+            });
+        }
+
+        private CompletableFuture<Void> deleteSnapshotSegment(PositionImpl positionNeedToDelete) {
+            long sequenceIdNeedToDelete = indexes.get(positionNeedToDelete).getSequenceID();
+            return snapshotSegmentsWriterFuture.thenCompose(writer -> writer.deleteAsync(buildKey(sequenceIdNeedToDelete), null))
+                    .thenRun(() -> {
+                        if (log.isDebugEnabled()) {
+                            log.debug("[{}] Successes to delete the snapshot segment, "
+                                            + "whose sequenceId is [{}] and maxReadPosition is [{}]",
+                                    this.topic.getName(), this.sequenceID, positionNeedToDelete);
+                        }
+                        abortTxnSegments.remove(positionNeedToDelete);
+                        //The process will check whether the snapshot segment is null, and update index when recovered.
+                        indexes.remove(positionNeedToDelete);
+                        //Keep index snapshot and update index
+                        updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                indexes.values().stream().toList());
+                    }).exceptionally(e -> {
+                        log.warn("[{}] Failed to delete the snapshot segment, "
+                                        + "whose sequenceId is [{}] and maxReadPosition is [{}]",
+                                this.topic.getName(), this.sequenceID, positionNeedToDelete, e);
+                        return null;
+                    });
+        }
+
+        //Update the indexes with the giving index snapshot and indexlist in the transactionBufferSnapshotIndexe.
+        private CompletableFuture<Void> updateSnapshotIndex(TransactionBufferSnapshotIndexesMetadata snapshotSegment,
+                                                            List<TransactionBufferSnapshotIndex> indexList) {

Review Comment:
   All the accesses to the variable `indexes` are running in the same thread, right? So we shouldn't need this `args[indexList]`, because we can access the variable `indexes` here.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()

Review Comment:
   If the `ceilingKey` method returns null, does that mean that all segments does not matches, so we don't need to loop `abortTxnSegments`, right?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    private void updateSnapshotIndexMetadataByChangeTimes() {
+        if (this.changeMaxReadPositionAndAddAbortTimes.incrementAndGet() == takeSnapshotIntervalNumber) {
+            changeMaxReadPositionAndAddAbortTimes.set(0);
+            persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                    persistentWorker::updateIndexMetadataForTheLastSnapshot);
+        }
+    }
+
+    private void takeSnapshotByTimeout() {

Review Comment:
   suggestion method name `scheduledTriggerSnapshot`. By the way, why not merge `run` and `takeSnapshotByTimeout` into one method?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();

Review Comment:
   Why not delete all expired items?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);

Review Comment:
   suggestion use `ceilingEntry` instead of `ceilingKey`



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        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 -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @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.
+                        return FutureUtil.waitForAll(completableFutures).thenCompose((ignore) -> {
+                            if (invalidIndex.get() != 0 ) {
+                                persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                        -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                        indexes.values().stream().toList()));
+                            }
+                            return CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                        });
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public PositionImpl getMaxReadPosition() {
+        return this.maxReadPosition;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return persistentWorker.closeAsync();
+    }
+
+    private void handleSnapshotSegmentEntry(Entry entry) {
+        //decode snapshot from entry
+        ByteBuf headersAndPayload = entry.getDataBuffer();
+        //skip metadata
+        Commands.parseMessageMetadata(headersAndPayload);
+        TransactionBufferSnapshotSegment snapshotSegment = Schema.AVRO(TransactionBufferSnapshotSegment.class)
+                .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer());
+        abortTxnSegments.put(new PositionImpl(snapshotSegment.getMaxReadPositionLedgerId(),
+                snapshotSegment.getMaxReadPositionEntryId()), deserializationFotSnapshotSegment(
+                snapshotSegment.getAborts()));
+
+    }
+
+    private <T> void  closeReader(SystemTopicClient.Reader<T> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private class PersistentWorker {
+        protected final AtomicLong sequenceID = new AtomicLong(0);
+
+        private final PersistentTopic topic;
+
+        //Persistent snapshot segment and index at the single thread.
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+                snapshotSegmentsWriterFuture;
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+                snapshotIndexWriterFuture;
+
+        private enum OperationState {
+            None,
+            UpdatingIndex,
+            WritingSegment,
+            DeletingSegment,
+            Closing,
+            Closed
+        }
+        private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState>
+                STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class,
+                        PersistentWorker.OperationState.class, "operationState");
+
+        public enum OperationType {
+            UpdateIndex,
+            WriteSegment,
+            DeleteSegment,
+            Close
+        }
+
+        private volatile OperationState operationState = OperationState.None;
+
+        ConcurrentLinkedDeque<Pair<OperationType, Supplier<CompletableFuture<Void>>>> taskQueue =
+                new ConcurrentLinkedDeque<>();
+        private CompletableFuture<Void> lastOperationFuture;
+        private final Timer timer;
+
+        public PersistentWorker(PersistentTopic topic) {
+            this.topic = topic;
+            this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+            this.snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+            this.snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        }
+
+        public void appendTask(OperationType operationType, Supplier<CompletableFuture<Void>> task) {
+            switch (operationType) {
+                case UpdateIndex -> {
+                    if (!taskQueue.isEmpty()) {
+                        return;
+                    } else if(STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.whenComplete((ignore, throwable) -> {
+                            if (throwable != null && log.isDebugEnabled()) {
+                                log.debug("[{}] Failed to update index snapshot", topic.getName(), throwable);
+                            }
+
+                            STATE_UPDATER.compareAndSet(this, OperationState.UpdatingIndex, OperationState.None);
+                        });
+                    }
+                }
+                case WriteSegment, DeleteSegment -> {
+                    taskQueue.add(new MutablePair<>(operationType, task));
+                    executeTask();
+                }
+                case Close -> {
+                    STATE_UPDATER.set(this, OperationState.Closing);
+                    taskQueue.clear();
+                    lastOperationFuture.thenRun(() -> {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.thenRun(() ->
+                                STATE_UPDATER.compareAndSet(this, OperationState.Closing, OperationState.Closed));
+                    });
+                }
+            }
+        }
+
+        private void executeTask() {
+            OperationType operationType = taskQueue.getFirst().getKey();
+            switch (operationType) {
+                case WriteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.WritingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.WriteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to write snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+                                STATE_UPDATER.compareAndSet(this, OperationState.WritingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+                case DeleteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.DeletingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.DeleteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to delete snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+
+                                STATE_UPDATER.compareAndSet(this, OperationState.DeletingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+            }
+        }
+
+        protected CompletableFuture<Void> takeSnapshotSegmentAsync(ArrayList<TxnID> sealedAbortedTxnIdSegment,
+                                                                 PositionImpl maxReadPosition) {
+            return writeSnapshotSegmentAsync(sealedAbortedTxnIdSegment, maxReadPosition).thenRun(() -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Successes to take snapshot segment [{}] at maxReadPosition [{}] "
+                                    + "for the topic [{}], and the size of the segment is [{}]",
+                            this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size());
+                }
+                this.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 [{}]",
+                        this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size(), e);
+                return null;
+            });
+        }
+
+        private CompletableFuture<Void> writeSnapshotSegmentAsync(List<TxnID> segment, PositionImpl maxReadPosition) {
+            TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment();
+            transactionBufferSnapshotSegment.setAborts(serializationForSegment(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);

Review Comment:
   The method `thenCompose` executed above cause `indexes.put(maxReadPosition, index)` will be executed in another thread, which makes concurrent access `indexes`. We should use `thenApply` instead `thenCompose`.
   
   Same for `deleteSnapshotSegment`, `updateIndexMetadataForTheLastSnapshot`
   
   Same for variable `persistentSnapshotIndexes`



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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008984864


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,77 @@
+/**
+ * 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 java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.pulsar.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * After the transaction buffer writes a transaction aborted marker to the topic,
+     * the transaction buffer will put the aborted txnID and the aborted marker position to AbortedTxnProcessor.
+     * @param txnID aborted transaction ID.
+     * @param position the position of the abort txnID
+     */
+    void putAbortedTxnAndPosition(TxnID txnID, PositionImpl position);
+
+    /**
+     * Clean up invalid aborted transactions.
+     */
+    void trimExpiredAbortedTxns();
+
+    /**
+     * 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(TxnID txnID, Position readPosition);
+
+    /**
+     * Recover transaction buffer by transaction buffer snapshot.
+     * @return a Position (startReadCursorPosition) determiner where to start to recover in the original topic.
+     */
+
+    CompletableFuture<PositionImpl> recoverFromSnapshot();
+
+    /**
+     * Delete the transaction buffer aborted transaction snapshot.
+     * @return a completableFuture.
+     */
+    CompletableFuture<Void> deleteAbortedTxnSnapshot();
+
+    /**
+     * Take the frist snapshot if the topic has no snapshot before.

Review Comment:
   Take aborted transactions snapshot.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+    }
+
+    @Override
+    public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    //In this implementation we clear the invalid aborted txn ID one by one.
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshot> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
+                                if (transactionBufferSnapshot != null) {
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> deleteAbortedTxnSnapshot() {
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenRun(this::closeAsync);

Review Comment:
   this method don't need close



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;

Review Comment:
   delete



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,77 @@
+/**
+ * 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 java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.pulsar.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * After the transaction buffer writes a transaction aborted marker to the topic,
+     * the transaction buffer will put the aborted txnID and the aborted marker position to AbortedTxnProcessor.
+     * @param txnID aborted transaction ID.
+     * @param position the position of the abort txnID
+     */
+    void putAbortedTxnAndPosition(TxnID txnID, PositionImpl position);
+
+    /**
+     * Clean up invalid aborted transactions.
+     */
+    void trimExpiredAbortedTxns();
+
+    /**
+     * 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(TxnID txnID, Position readPosition);
+
+    /**
+     * Recover transaction buffer by transaction buffer snapshot.
+     * @return a Position (startReadCursorPosition) determiner where to start to recover in the original topic.
+     */
+
+    CompletableFuture<PositionImpl> recoverFromSnapshot();
+
+    /**
+     * Delete the transaction buffer aborted transaction snapshot.
+     * @return a completableFuture.
+     */
+    CompletableFuture<Void> deleteAbortedTxnSnapshot();
+
+    /**
+     * Take the frist snapshot if the topic has no snapshot before.
+     * @return a completableFuture.
+     */
+    CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition);

Review Comment:
   ```suggestion
       CompletableFuture<Void> takeAbortedTxnsSnapshot(PositionImpl maxReadPosition);
   ```



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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1007677796


##########
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:
   Yes, you are right.



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


[GitHub] [pulsar] codecov-commenter commented on pull request #17847: [feat][txn] Implement the AbortedTxnProcessor for TransactionBuffer

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#issuecomment-1297330407

   # [Codecov](https://codecov.io/gh/apache/pulsar/pull/17847?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#17847](https://codecov.io/gh/apache/pulsar/pull/17847?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (7b1b817) into [master](https://codecov.io/gh/apache/pulsar/commit/0866c3a6a734b39402a6bc8349bab13edab00488?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0866c3a) will **increase** coverage by `31.81%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/pulsar/pull/17847/graphs/tree.svg?width=650&height=150&src=pr&token=acYqCpsK9J&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/pulsar/pull/17847?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #17847       +/-   ##
   =============================================
   + Coverage     38.97%   70.79%   +31.81%     
   + Complexity     8311      438     -7873     
   =============================================
     Files           683       26      -657     
     Lines         67325     2246    -65079     
     Branches       7217      245     -6972     
   =============================================
   - Hits          26239     1590    -24649     
   + Misses        38079      483    -37596     
   + Partials       3007      173     -2834     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests | `70.79% <ø> (+31.81%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pulsar/pull/17847?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache/pulsar/proxy/server/LookupProxyHandler.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLXByb3h5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9wdWxzYXIvcHJveHkvc2VydmVyL0xvb2t1cFByb3h5SGFuZGxlci5qYXZh) | `57.75% <0.00%> (-0.44%)` | :arrow_down: |
   | [.../pulsar/broker/service/AbstractBaseDispatcher.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL0Fic3RyYWN0QmFzZURpc3BhdGNoZXIuamF2YQ==) | | |
   | [...sar/broker/service/persistent/PersistentTopic.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci9zZXJ2aWNlL3BlcnNpc3RlbnQvUGVyc2lzdGVudFRvcGljLmphdmE=) | | |
   | [...ransaction/buffer/impl/InMemTransactionBuffer.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci90cmFuc2FjdGlvbi9idWZmZXIvaW1wbC9Jbk1lbVRyYW5zYWN0aW9uQnVmZmVyLmphdmE=) | | |
   | [...ransaction/buffer/impl/TopicTransactionBuffer.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci90cmFuc2FjdGlvbi9idWZmZXIvaW1wbC9Ub3BpY1RyYW5zYWN0aW9uQnVmZmVyLmphdmE=) | | |
   | [...nsaction/buffer/impl/TransactionBufferDisable.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci90cmFuc2FjdGlvbi9idWZmZXIvaW1wbC9UcmFuc2FjdGlvbkJ1ZmZlckRpc2FibGUuamF2YQ==) | | |
   | [.../metadata/v2/TransactionBufferSnapshotIndexes.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWJyb2tlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2Jyb2tlci90cmFuc2FjdGlvbi9idWZmZXIvbWV0YWRhdGEvdjIvVHJhbnNhY3Rpb25CdWZmZXJTbmFwc2hvdEluZGV4ZXMuamF2YQ==) | | |
   | [...ad/jcloud/impl/OffloadIndexBlockV2BuilderImpl.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-dGllcmVkLXN0b3JhZ2UvamNsb3VkL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9ib29ra2VlcGVyL21sZWRnZXIvb2ZmbG9hZC9qY2xvdWQvaW1wbC9PZmZsb2FkSW5kZXhCbG9ja1YyQnVpbGRlckltcGwuamF2YQ==) | | |
   | [...apache/bookkeeper/mledger/ManagedLedgerConfig.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-bWFuYWdlZC1sZWRnZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvbWxlZGdlci9NYW5hZ2VkTGVkZ2VyQ29uZmlnLmphdmE=) | | |
   | [...apache/pulsar/client/impl/ProducerBuilderImpl.java](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHVsc2FyLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NsaWVudC9pbXBsL1Byb2R1Y2VyQnVpbGRlckltcGwuamF2YQ==) | | |
   | ... and [648 more](https://codecov.io/gh/apache/pulsar/pull/17847/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   


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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1007856564


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,99 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+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(TxnID 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);

Review Comment:
   If use this method, TopicTransactionBuffer is in NoSnapshot state. If TopicTransactionBuffer in NoSnapshot state, we don't take any snapshot right? so we only use updateMaxReadPosition is enough, right?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -625,108 +520,76 @@ public static class TopicTransactionBufferRecover implements Runnable {
 
         private final TopicTransactionBuffer topicTransactionBuffer;
 
-        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+        private final AbortedTxnProcessor abortedTxnProcessor;
 
         private TopicTransactionBufferRecover(TopicTransactionBufferRecoverCallBack callBack, PersistentTopic topic,
-                                              TopicTransactionBuffer transactionBuffer, CompletableFuture<
-                SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter) {
+                                              TopicTransactionBuffer transactionBuffer,
+                                              AbortedTxnProcessor abortedTxnProcessor) {
             this.topic = topic;
             this.callBack = callBack;
             this.entryQueue = new SpscArrayQueue<>(2000);
             this.topicTransactionBuffer = transactionBuffer;
-            this.takeSnapshotWriter = takeSnapshotWriter;
+            this.abortedTxnProcessor = abortedTxnProcessor;
         }
 
         @SneakyThrows
         @Override
         public void run() {
-            this.takeSnapshotWriter.thenRunAsync(() -> {
-                if (!this.topicTransactionBuffer.changeToInitializingState()) {
-                    log.warn("TransactionBuffer {} of topic {} can not change state to Initializing",
-                            this, topic.getName());
+            if (!this.topicTransactionBuffer.changeToInitializingState()) {
+                log.warn("TransactionBuffer {} of topic {} can not change state to Initializing",
+                        this, topic.getName());
+                return;
+            }
+            abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> {
+                //Transaction is not enable for this topic, so just make maxReadPosition as LAC.

Review Comment:
   // if startReadCursorPosition is null, it means that this topic has not sent a transaction message, the TopicTransactionBuffer state is NoSnapshot, so we do not need to perform the recovery process from the original topic
   



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -383,11 +347,9 @@ public CompletableFuture<Void> abortTxn(TxnID txnID, long lowWaterMark) {
                     @Override
                     public void addComplete(Position position, ByteBuf entryData, Object ctx) {
                         synchronized (TopicTransactionBuffer.this) {
-                            aborts.put(txnID, (PositionImpl) position);
-                            updateMaxReadPosition(txnID);
-                            changeMaxReadPositionAndAddAbortTimes.getAndIncrement();
-                            clearAbortedTransactions();
-                            takeSnapshotByChangeTimes();
+                            PositionImpl maxReadPosition = updateMaxReadPosition(txnID);
+                            snapshotAbortedTxnProcessor.appendAbortedTxn(txnID, maxReadPosition);

Review Comment:
   These two ops must guarantee atomicity with take snapshot, otherwise, transient may be wrong



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,255 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position maxReadPosition) {
+        if (this.maxReadPosition != maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) maxReadPosition;
+            takeSnapshotByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                aborts.firstKey();
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    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;
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasSnapshot) {
+                            return CompletableFuture.completedFuture(null);
+                        }
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenCompose(__ -> CompletableFuture.completedFuture(null));
+    }
+
+    @Override
+    public CompletableFuture<Void> takesFirstSnapshot() {
+        return takeSnapshot();
+    }
+
+    @Override
+    public PositionImpl getMaxReadPosition() {
+        return maxReadPosition;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return takeSnapshotWriter.thenCompose(SystemTopicClient.Writer::closeAsync);
+    }
+
+    private void closeReader(SystemTopicClient.Reader<TransactionBufferSnapshot> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private void takeSnapshotByChangeTimes() {
+        if (changeMaxReadPositionAndAddAbortTimes.incrementAndGet() >= takeSnapshotIntervalNumber) {
+            takeSnapshot();
+        }
+    }
+
+    private void takeSnapshotByTimeout() {
+        if (changeMaxReadPositionAndAddAbortTimes.get() > 0) {
+            takeSnapshot();
+        }
+        this.timer.newTimeout(SingleSnapshotAbortedTxnProcessorImpl.this,
+                takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+    }
+
+    private void handleSnapshot(TransactionBufferSnapshot snapshot) {
+        maxReadPosition = PositionImpl.get(snapshot.getMaxReadPositionLedgerId(),
+                snapshot.getMaxReadPositionEntryId());
+        if (snapshot.getAborts() != null) {
+            snapshot.getAborts().forEach(abortTxnMetadata ->
+                    aborts.put(new TxnID(abortTxnMetadata.getTxnIdMostBits(),
+                                    abortTxnMetadata.getTxnIdLeastBits()),
+                            PositionImpl.get(abortTxnMetadata.getLedgerId(),
+                                    abortTxnMetadata.getEntryId())));
+        }
+    }
+
+    private CompletableFuture<Void> takeSnapshot() {
+        changeMaxReadPositionAndAddAbortTimes.set(0);
+        return takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            synchronized (SingleSnapshotAbortedTxnProcessorImpl.this) {

Review Comment:
   the orignal lock is TopicTransactionBuffer.this, if use processer to lock, aborts and maxReadPosition may not consistent



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,672 @@
+/**
+ * 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.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.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        this.maxReadPosition = new PositionImpl(persistentSnapshotIndexes
+                                .getSnapshot().getMaxReadPositionLedgerId(),
+                                persistentSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                        if (indexes.size() != 0) {
+                            persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);

Review Comment:
   If index.size() = 0, the sequenceID may not 0, we need to think about how to get the correct sequnceID, may it not the sequenceID, it is segementID is right



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,672 @@
+/**
+ * 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.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.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        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 -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @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.
+                        return FutureUtil.waitForAll(completableFutures).thenCompose((ignore) -> {
+                            if (invalidIndex.get() != 0 ) {
+                                persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                        -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                        indexes.values().stream().toList()));
+                            }
+                            return CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                        });
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public PositionImpl getMaxReadPosition() {
+        return this.maxReadPosition;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return persistentWorker.closeAsync();
+    }
+
+    private void handleSnapshotSegmentEntry(Entry entry) {
+        //decode snapshot from entry
+        ByteBuf headersAndPayload = entry.getDataBuffer();
+        //skip metadata
+        Commands.parseMessageMetadata(headersAndPayload);
+        TransactionBufferSnapshotSegment snapshotSegment = Schema.AVRO(TransactionBufferSnapshotSegment.class)
+                .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer());
+        abortTxnSegments.put(new PositionImpl(snapshotSegment.getMaxReadPositionLedgerId(),
+                snapshotSegment.getMaxReadPositionEntryId()), deserializationFotSnapshotSegment(
+                snapshotSegment.getAborts()));
+
+    }
+
+    private <T> void  closeReader(SystemTopicClient.Reader<T> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private class PersistentWorker {
+        protected final AtomicLong sequenceID = new AtomicLong(0);
+
+        private final PersistentTopic topic;
+
+        //Persistent snapshot segment and index at the single thread.
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+                snapshotSegmentsWriterFuture;
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+                snapshotIndexWriterFuture;
+
+        private enum OperationState {
+            None,
+            UpdatingIndex,
+            WritingSegment,
+            DeletingSegment,
+            Closing,
+            Closed
+        }
+        private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState>
+                STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class,
+                        PersistentWorker.OperationState.class, "operationState");
+
+        public enum OperationType {
+            UpdateIndex,
+            WriteSegment,
+            DeleteSegment,
+            Close
+        }
+
+        private volatile OperationState operationState = OperationState.None;
+
+        ConcurrentSkipListMap<OperationType, Supplier<CompletableFuture<Void>>> taskQueue =

Review Comment:
   why use skipListMp? the op will not in order



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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008807202


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;

Review Comment:
   can delete right?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * 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(TxnID txnID, PositionImpl position);
+
+    /**
+     * 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.

Review Comment:
   Clean up invalid aborted transactions.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);

Review Comment:
   If it does not exist in the segment, we directly check unsealed segment is ok



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshot> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
+                                if (transactionBufferSnapshot != null) {
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {

Review Comment:
   This method is only for deleting snapshots, it doesn't need close right?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * After the transaction buffer writes a transaction aborted mark to the topic,
+     * the transaction buffer will add the aborted transaction ID to AbortedTxnProcessor.

Review Comment:
   ```suggestion
        * After the transaction buffer writes a transaction aborted marker to the topic, the transaction buffer will put the aborted txnID and the aborted marker position to AbortedTxnProcessor.
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();

Review Comment:
   also can delete



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * 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(TxnID txnID, PositionImpl position);
+
+    /**
+     * 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 trimExpiredAbortedTxns();
+
+    /**
+     * 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(TxnID 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.

Review Comment:
   only return the position, not use the pair



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * 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.

Review Comment:
   ```suggestion
        * @param txnID aborted transaction ID.
        * @param position the position of the abort txnID
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshot> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
+                                if (transactionBufferSnapshot != null) {
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenRun(this::closeAsync);
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, aborts);
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return takeSnapshotWriter.thenCompose(SystemTopicClient.Writer::closeAsync);
+    }
+
+    private void closeReader(SystemTopicClient.Reader<TransactionBufferSnapshot> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private void handleSnapshot(TransactionBufferSnapshot snapshot) {
+        maxReadPosition = PositionImpl.get(snapshot.getMaxReadPositionLedgerId(),
+                snapshot.getMaxReadPositionEntryId());
+        if (snapshot.getAborts() != null) {
+            snapshot.getAborts().forEach(abortTxnMetadata ->
+                    aborts.put(new TxnID(abortTxnMetadata.getTxnIdMostBits(),
+                                    abortTxnMetadata.getTxnIdLeastBits()),
+                            PositionImpl.get(abortTxnMetadata.getLedgerId(),
+                                    abortTxnMetadata.getEntryId())));
+        }
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, LinkedMap<TxnID, PositionImpl> aborts) {
+        changeMaxReadPositionAndAddAbortTimes.set(0);
+        return takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            synchronized (topic) {
+                snapshot.setTopicName(topic.getName());
+                snapshot.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+                snapshot.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+                List<AbortTxnMetadata> list = new ArrayList<>();
+                aborts.forEach((k, v) -> {
+                    AbortTxnMetadata abortTxnMetadata = new AbortTxnMetadata();
+                    abortTxnMetadata.setTxnIdMostBits(k.getMostSigBits());
+                    abortTxnMetadata.setTxnIdLeastBits(k.getLeastSigBits());
+                    abortTxnMetadata.setLedgerId(v.getLedgerId());
+                    abortTxnMetadata.setEntryId(v.getEntryId());
+                    list.add(abortTxnMetadata);
+                });
+                snapshot.setAborts(list);
+            }
+            return writer.writeAsync(snapshot.getTopicName(), snapshot).thenAccept(messageId -> {
+                this.lastSnapshotTimestamps = System.currentTimeMillis();
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}]Transaction buffer take snapshot success! "
+                            + "messageId : {}", topic.getName(), messageId);
+                }
+            }).exceptionally(e -> {
+                log.warn("[{}]Transaction buffer take snapshot fail! ", topic.getName(), e);

Review Comment:
   ```suggestion
                   log.warn("[{}]Transaction buffer take snapshot fail! ", topic.getName(), e.getCause());
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * 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(TxnID txnID, PositionImpl position);
+
+    /**
+     * 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.

Review Comment:
   ```suggestion
   ```
   you can add the note in the implementation class method



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * 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(TxnID txnID, PositionImpl position);

Review Comment:
   ```suggestion
       void putAbortedTxnAndPosition(TxnID txnID, PositionImpl position);
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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;

Review Comment:
   delete this import



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshot> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
+                                if (transactionBufferSnapshot != null) {
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenRun(this::closeAsync);
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, aborts);
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return takeSnapshotWriter.thenCompose(SystemTopicClient.Writer::closeAsync);
+    }
+
+    private void closeReader(SystemTopicClient.Reader<TransactionBufferSnapshot> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private void handleSnapshot(TransactionBufferSnapshot snapshot) {
+        maxReadPosition = PositionImpl.get(snapshot.getMaxReadPositionLedgerId(),
+                snapshot.getMaxReadPositionEntryId());
+        if (snapshot.getAborts() != null) {
+            snapshot.getAborts().forEach(abortTxnMetadata ->
+                    aborts.put(new TxnID(abortTxnMetadata.getTxnIdMostBits(),
+                                    abortTxnMetadata.getTxnIdLeastBits()),
+                            PositionImpl.get(abortTxnMetadata.getLedgerId(),
+                                    abortTxnMetadata.getEntryId())));
+        }
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, LinkedMap<TxnID, PositionImpl> aborts) {
+        changeMaxReadPositionAndAddAbortTimes.set(0);

Review Comment:
   delete



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,605 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,2
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();

Review Comment:
   checkAbortTxn uses different threads, we should use thread-safe collections. We use `ConcurrentOpenHashSet`, because if we use a list, it will be traversed during a query, affecting performance 



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshot> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
+                                if (transactionBufferSnapshot != null) {
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenRun(this::closeAsync);
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, aborts);
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return takeSnapshotWriter.thenCompose(SystemTopicClient.Writer::closeAsync);
+    }
+
+    private void closeReader(SystemTopicClient.Reader<TransactionBufferSnapshot> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private void handleSnapshot(TransactionBufferSnapshot snapshot) {
+        maxReadPosition = PositionImpl.get(snapshot.getMaxReadPositionLedgerId(),
+                snapshot.getMaxReadPositionEntryId());
+        if (snapshot.getAborts() != null) {
+            snapshot.getAborts().forEach(abortTxnMetadata ->
+                    aborts.put(new TxnID(abortTxnMetadata.getTxnIdMostBits(),
+                                    abortTxnMetadata.getTxnIdLeastBits()),
+                            PositionImpl.get(abortTxnMetadata.getLedgerId(),
+                                    abortTxnMetadata.getEntryId())));
+        }
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, LinkedMap<TxnID, PositionImpl> aborts) {
+        changeMaxReadPositionAndAddAbortTimes.set(0);
+        return takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            synchronized (topic) {
+                snapshot.setTopicName(topic.getName());
+                snapshot.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
+                snapshot.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
+                List<AbortTxnMetadata> list = new ArrayList<>();
+                aborts.forEach((k, v) -> {
+                    AbortTxnMetadata abortTxnMetadata = new AbortTxnMetadata();
+                    abortTxnMetadata.setTxnIdMostBits(k.getMostSigBits());
+                    abortTxnMetadata.setTxnIdLeastBits(k.getLeastSigBits());
+                    abortTxnMetadata.setLedgerId(v.getLedgerId());
+                    abortTxnMetadata.setEntryId(v.getEntryId());
+                    list.add(abortTxnMetadata);
+                });
+                snapshot.setAborts(list);
+            }

Review Comment:
   ```suggestion
               TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
               snapshot.setTopicName(topic.getName());
               snapshot.setMaxReadPositionLedgerId(maxReadPosition.getLedgerId());
               snapshot.setMaxReadPositionEntryId(maxReadPosition.getEntryId());
               List<AbortTxnMetadata> list = new ArrayList<>();
               aborts.forEach((k, v) -> {
                   AbortTxnMetadata abortTxnMetadata = new AbortTxnMetadata();
                   abortTxnMetadata.setTxnIdMostBits(k.getMostSigBits());
                   abortTxnMetadata.setTxnIdLeastBits(k.getLeastSigBits());
                   abortTxnMetadata.setLedgerId(v.getLedgerId());
                   abortTxnMetadata.setEntryId(v.getEntryId());
                   list.add(abortTxnMetadata);
               });
               snapshot.setAborts(list);
           return takeSnapshotWriter.thenCompose(writer -> {
    
               }
   ```
   
   in this way, we just need to make `takeAbortedTxnSnapshot` and updateMaxReadPosition and putAborts thread-safe is  OK



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java:
##########
@@ -0,0 +1,80 @@
+/**
+ * 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.client.api.transaction.TxnID;
+
+
+public interface AbortedTxnProcessor {
+
+    /**
+     * 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(TxnID txnID, PositionImpl position);
+
+    /**
+     * 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 trimExpiredAbortedTxns();
+
+    /**
+     * 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(TxnID 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();
+
+    /**
+     * Clear the snapshot/snapshot segment and index for this topic.
+     * @return a completableFuture.
+     */
+    CompletableFuture<Void> clearAndCloseAsync();

Review Comment:
   ```suggestion
        * Delete the transaction buffer aborted transaction snapshot.
        * @return a completableFuture.
        */
       CompletableFuture<Void> deleteAbortedTxnSnapshot();
   ```



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,204 @@
+/**
+ * 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.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    try {
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshot> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshot transactionBufferSnapshot = message.getValue();
+                                if (transactionBufferSnapshot != null) {
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenRun(this::closeAsync);
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, aborts);
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return takeSnapshotWriter.thenCompose(SystemTopicClient.Writer::closeAsync);
+    }
+
+    private void closeReader(SystemTopicClient.Reader<TransactionBufferSnapshot> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private void handleSnapshot(TransactionBufferSnapshot snapshot) {
+        maxReadPosition = PositionImpl.get(snapshot.getMaxReadPositionLedgerId(),
+                snapshot.getMaxReadPositionEntryId());
+        if (snapshot.getAborts() != null) {
+            snapshot.getAborts().forEach(abortTxnMetadata ->
+                    aborts.put(new TxnID(abortTxnMetadata.getTxnIdMostBits(),
+                                    abortTxnMetadata.getTxnIdLeastBits()),
+                            PositionImpl.get(abortTxnMetadata.getLedgerId(),
+                                    abortTxnMetadata.getEntryId())));
+        }
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, LinkedMap<TxnID, PositionImpl> aborts) {

Review Comment:
   move to `public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {`



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,605 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,2
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments

Review Comment:
   ```suggestion
       private final ConcurrentSkipListMap<PositionImpl, Set<TxnID>> abortTxnSegments
   ```



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


[GitHub] [pulsar] congbobo184 closed pull request #17847: [feat][txn] Implement the AbortedTxnProcessor for TransactionBuffer

Posted by GitBox <gi...@apache.org>.
congbobo184 closed pull request #17847: [feat][txn] Implement the AbortedTxnProcessor for TransactionBuffer
URL: https://github.com/apache/pulsar/pull/17847


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


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

Posted by GitBox <gi...@apache.org>.
poorbarcode commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008715721


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,600 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final Timer timer;
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            abortTxnSegments.put(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, unsealedAbortedTxnIdSegment);
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, ArrayList<TxnID> aborts) {

Review Comment:
   Why we need two method `takeAbortedTxnSnapshot(PositionImpl)` and `takeAbortedTxnSnapshot(PositionImpl, List)`



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,600 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final Timer timer;
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            abortTxnSegments.put(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, unsealedAbortedTxnIdSegment);
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, ArrayList<TxnID> aborts) {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                () -> persistentWorker
+                        .updateIndexMetadataForTheLastSnapshot(maxReadPosition, aborts)
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        if (indexes.size() != 0) {
+                            persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                        }
+                    }
+                    //Read snapshot segment to recover aborts.
+                    ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+                    AtomicLong invalidIndex = new AtomicLong(0);
+                    AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback = new AsyncCallbacks
+                            .OpenReadOnlyManagedLedgerCallback() {
+                        @Override
+                        public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) {
+                            persistentSnapshotIndexes.getIndexList().forEach(index -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @Override
+                            public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) {
+                                //

Review Comment:
   Do exceptions need to be handled here?



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

Review Comment:
   Should we close timer task here?



##########
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:
   > We can not do this, because this system topic is shared by all topics in a namespace
   Got it.
   
   Should we deserialize only the last entry here?



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,675 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        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 -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @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.
+                        return FutureUtil.waitForAll(completableFutures).thenCompose((ignore) -> {
+                            if (invalidIndex.get() != 0 ) {
+                                persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                        -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                        indexes.values().stream().toList()));
+                            }
+                            return CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                        });
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public PositionImpl getMaxReadPosition() {
+        return this.maxReadPosition;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return persistentWorker.closeAsync();
+    }
+
+    private void handleSnapshotSegmentEntry(Entry entry) {
+        //decode snapshot from entry
+        ByteBuf headersAndPayload = entry.getDataBuffer();
+        //skip metadata
+        Commands.parseMessageMetadata(headersAndPayload);
+        TransactionBufferSnapshotSegment snapshotSegment = Schema.AVRO(TransactionBufferSnapshotSegment.class)
+                .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer());
+        abortTxnSegments.put(new PositionImpl(snapshotSegment.getMaxReadPositionLedgerId(),
+                snapshotSegment.getMaxReadPositionEntryId()), deserializationFotSnapshotSegment(
+                snapshotSegment.getAborts()));
+
+    }
+
+    private <T> void  closeReader(SystemTopicClient.Reader<T> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private class PersistentWorker {
+        protected final AtomicLong sequenceID = new AtomicLong(0);
+
+        private final PersistentTopic topic;
+
+        //Persistent snapshot segment and index at the single thread.
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+                snapshotSegmentsWriterFuture;
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+                snapshotIndexWriterFuture;
+
+        private enum OperationState {
+            None,
+            UpdatingIndex,
+            WritingSegment,
+            DeletingSegment,
+            Closing,
+            Closed
+        }
+        private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState>
+                STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class,
+                        PersistentWorker.OperationState.class, "operationState");
+
+        public enum OperationType {
+            UpdateIndex,
+            WriteSegment,
+            DeleteSegment,
+            Close
+        }
+
+        private volatile OperationState operationState = OperationState.None;
+
+        ConcurrentLinkedDeque<Pair<OperationType, Supplier<CompletableFuture<Void>>>> taskQueue =
+                new ConcurrentLinkedDeque<>();
+        private CompletableFuture<Void> lastOperationFuture;
+        private final Timer timer;
+
+        public PersistentWorker(PersistentTopic topic) {
+            this.topic = topic;
+            this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+            this.snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+            this.snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        }
+
+        public void appendTask(OperationType operationType, Supplier<CompletableFuture<Void>> task) {
+            switch (operationType) {
+                case UpdateIndex -> {
+                    if (!taskQueue.isEmpty()) {
+                        return;
+                    } else if(STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.whenComplete((ignore, throwable) -> {
+                            if (throwable != null && log.isDebugEnabled()) {
+                                log.debug("[{}] Failed to update index snapshot", topic.getName(), throwable);
+                            }
+
+                            STATE_UPDATER.compareAndSet(this, OperationState.UpdatingIndex, OperationState.None);
+                        });
+                    }
+                }
+                case WriteSegment, DeleteSegment -> {
+                    taskQueue.add(new MutablePair<>(operationType, task));
+                    executeTask();
+                }
+                case Close -> {
+                    STATE_UPDATER.set(this, OperationState.Closing);
+                    taskQueue.clear();
+                    lastOperationFuture.thenRun(() -> {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.thenRun(() ->
+                                STATE_UPDATER.compareAndSet(this, OperationState.Closing, OperationState.Closed));
+                    });
+                }
+            }
+        }
+
+        private void executeTask() {
+            OperationType operationType = taskQueue.getFirst().getKey();
+            switch (operationType) {
+                case WriteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.WritingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.WriteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to write snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+                                STATE_UPDATER.compareAndSet(this, OperationState.WritingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+                case DeleteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.DeletingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.DeleteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to delete snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+
+                                STATE_UPDATER.compareAndSet(this, OperationState.DeletingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+            }
+        }
+
+        protected CompletableFuture<Void> takeSnapshotSegmentAsync(ArrayList<TxnID> sealedAbortedTxnIdSegment,
+                                                                 PositionImpl maxReadPosition) {
+            return writeSnapshotSegmentAsync(sealedAbortedTxnIdSegment, maxReadPosition).thenRun(() -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Successes to take snapshot segment [{}] at maxReadPosition [{}] "
+                                    + "for the topic [{}], and the size of the segment is [{}]",
+                            this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size());
+                }
+                this.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 [{}]",
+                        this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size(), e);
+                return null;
+            });
+        }
+
+        private CompletableFuture<Void> writeSnapshotSegmentAsync(List<TxnID> segment, PositionImpl maxReadPosition) {
+            TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment();
+            transactionBufferSnapshotSegment.setAborts(serializationForSegment(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(new TransactionBufferSnapshotIndexesMetadata(
+                        maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), new ArrayList<>()),
+                        indexes.values().stream().toList());
+            });
+        }
+
+        private CompletableFuture<Void> deleteSnapshotSegment(PositionImpl positionNeedToDelete) {
+            long sequenceIdNeedToDelete = indexes.get(positionNeedToDelete).getSequenceID();
+            return snapshotSegmentsWriterFuture.thenCompose(writer -> writer.deleteAsync(buildKey(sequenceIdNeedToDelete), null))
+                    .thenRun(() -> {
+                        if (log.isDebugEnabled()) {
+                            log.debug("[{}] Successes to delete the snapshot segment, "
+                                            + "whose sequenceId is [{}] and maxReadPosition is [{}]",
+                                    this.topic.getName(), this.sequenceID, positionNeedToDelete);
+                        }
+                        abortTxnSegments.remove(positionNeedToDelete);
+                        //The process will check whether the snapshot segment is null, and update index when recovered.
+                        indexes.remove(positionNeedToDelete);
+                        //Keep index snapshot and update index
+                        updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                indexes.values().stream().toList());
+                    }).exceptionally(e -> {
+                        log.warn("[{}] Failed to delete the snapshot segment, "
+                                        + "whose sequenceId is [{}] and maxReadPosition is [{}]",
+                                this.topic.getName(), this.sequenceID, positionNeedToDelete, e);
+                        return null;
+                    });
+        }
+
+        //Update the indexes with the giving index snapshot and indexlist in the transactionBufferSnapshotIndexe.
+        private CompletableFuture<Void> updateSnapshotIndex(TransactionBufferSnapshotIndexesMetadata snapshotSegment,
+                                                            List<TransactionBufferSnapshotIndex> indexList) {

Review Comment:
   I know why need this arg now. Please close this comment.



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,600 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final Timer timer;
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            abortTxnSegments.put(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, unsealedAbortedTxnIdSegment);
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, ArrayList<TxnID> aborts) {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                () -> persistentWorker
+                        .updateIndexMetadataForTheLastSnapshot(maxReadPosition, aborts)
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        if (indexes.size() != 0) {
+                            persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                        }
+                    }
+                    //Read snapshot segment to recover aborts.
+                    ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+                    AtomicLong invalidIndex = new AtomicLong(0);
+                    AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback = new AsyncCallbacks
+                            .OpenReadOnlyManagedLedgerCallback() {
+                        @Override
+                        public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, Object ctx) {
+                            persistentSnapshotIndexes.getIndexList().forEach(index -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @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.
+                        return FutureUtil.waitForAll(completableFutures).thenCompose((ignore) -> {
+                            if (invalidIndex.get() != 0 ) {
+                                persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                        -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                        indexes.values().stream().toList()));
+                            }
+                            return CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                        });
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return persistentWorker.closeAsync();
+    }
+
+    private void handleSnapshotSegmentEntry(Entry entry) {
+        //decode snapshot from entry
+        ByteBuf headersAndPayload = entry.getDataBuffer();
+        //skip metadata
+        Commands.parseMessageMetadata(headersAndPayload);
+        TransactionBufferSnapshotSegment snapshotSegment = Schema.AVRO(TransactionBufferSnapshotSegment.class)
+                .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer());
+        abortTxnSegments.put(new PositionImpl(snapshotSegment.getMaxReadPositionLedgerId(),
+                snapshotSegment.getMaxReadPositionEntryId()), deserializationFotSnapshotSegment(
+                snapshotSegment.getAborts()));
+
+    }
+
+    private <T> void  closeReader(SystemTopicClient.Reader<T> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private class PersistentWorker {
+        protected final AtomicLong sequenceID = new AtomicLong(0);
+
+        private final PersistentTopic topic;
+
+        //Persistent snapshot segment and index at the single thread.
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+                snapshotSegmentsWriterFuture;
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+                snapshotIndexWriterFuture;
+
+        private enum OperationState {
+            None,
+            UpdatingIndex,
+            WritingSegment,
+            DeletingSegment,
+            Closing,
+            Closed
+        }
+        private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState>
+                STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class,
+                        PersistentWorker.OperationState.class, "operationState");
+
+        public enum OperationType {
+            UpdateIndex,
+            WriteSegment,
+            DeleteSegment,
+            Close
+        }
+
+        private volatile OperationState operationState = OperationState.None;
+
+        ConcurrentLinkedDeque<Pair<OperationType, Supplier<CompletableFuture<Void>>>> taskQueue =
+                new ConcurrentLinkedDeque<>();
+        private CompletableFuture<Void> lastOperationFuture;
+        private final Timer timer;
+
+        public PersistentWorker(PersistentTopic topic) {
+            this.topic = topic;
+            this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+            this.snapshotSegmentsWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotSegmentService().createWriter(TopicName.get(topic.getName()));
+            this.snapshotIndexWriterFuture =  this.topic.getBrokerService().getPulsar()
+                    .getTransactionBufferSnapshotServiceFactory()
+                    .getTxnBufferSnapshotIndexService().createWriter(TopicName.get(topic.getName()));
+
+        }
+
+        public void appendTask(OperationType operationType, Supplier<CompletableFuture<Void>> task) {
+            switch (operationType) {
+                case UpdateIndex -> {
+                    if (!taskQueue.isEmpty()) {
+                        return;
+                    } else if(STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.UpdatingIndex)) {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.whenComplete((ignore, throwable) -> {
+                            if (throwable != null && log.isDebugEnabled()) {
+                                log.debug("[{}] Failed to update index snapshot", topic.getName(), throwable);
+                            }
+
+                            STATE_UPDATER.compareAndSet(this, OperationState.UpdatingIndex, OperationState.None);
+                        });
+                    }
+                }
+                case WriteSegment, DeleteSegment -> {
+                    taskQueue.add(new MutablePair<>(operationType, task));
+                    executeTask();
+                }
+                case Close -> {
+                    STATE_UPDATER.set(this, OperationState.Closing);
+                    taskQueue.clear();
+                    lastOperationFuture.thenRun(() -> {
+                        lastOperationFuture = task.get();
+                        lastOperationFuture.thenRun(() ->
+                                STATE_UPDATER.compareAndSet(this, OperationState.Closing, OperationState.Closed));
+                    });
+                }
+            }
+        }
+
+        private void executeTask() {
+            OperationType operationType = taskQueue.getFirst().getKey();
+            switch (operationType) {
+                case WriteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.WritingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.WriteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to write snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+                                STATE_UPDATER.compareAndSet(this, OperationState.WritingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+                case DeleteSegment -> {
+                    if (STATE_UPDATER.compareAndSet(this, OperationState.None, OperationState.DeletingSegment)) {
+                        if (taskQueue.getFirst().getKey() == OperationType.DeleteSegment) {
+                            lastOperationFuture = taskQueue.getFirst().getValue().get();
+                            lastOperationFuture.whenComplete((ignore, throwable) -> {
+                                if (throwable != null) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("[{}] Failed to delete snapshot segment", topic.getName(), throwable);
+                                    }
+                                    timer.newTimeout(timeout -> executeTask(),
+                                            takeSnapshotIntervalTime, TimeUnit.MILLISECONDS);
+                                } else {
+                                    taskQueue.removeFirst();
+                                }
+
+                                STATE_UPDATER.compareAndSet(this, OperationState.DeletingSegment, OperationState.None);
+                            });
+                        }
+                    }
+                }
+            }
+        }
+
+        protected CompletableFuture<Void> takeSnapshotSegmentAsync(ArrayList<TxnID> sealedAbortedTxnIdSegment,
+                                                                 PositionImpl maxReadPosition) {
+            return writeSnapshotSegmentAsync(sealedAbortedTxnIdSegment, maxReadPosition).thenRun(() -> {
+                if (log.isDebugEnabled()) {
+                    log.debug("Successes to take snapshot segment [{}] at maxReadPosition [{}] "
+                                    + "for the topic [{}], and the size of the segment is [{}]",
+                            this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size());
+                }
+                this.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 [{}]",
+                        this.sequenceID, maxReadPosition, topic.getName(), sealedAbortedTxnIdSegment.size(), e);
+                return null;
+            });
+        }
+
+        private CompletableFuture<Void> writeSnapshotSegmentAsync(List<TxnID> segment, PositionImpl maxReadPosition) {
+            TransactionBufferSnapshotSegment transactionBufferSnapshotSegment = new TransactionBufferSnapshotSegment();
+            transactionBufferSnapshotSegment.setAborts(serializationForSegment(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(new TransactionBufferSnapshotIndexesMetadata(
+                        maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), new ArrayList<>()),
+                        indexes.values().stream().toList());
+            });
+        }
+
+        private CompletableFuture<Void> deleteSnapshotSegment(PositionImpl positionNeedToDelete) {
+            long sequenceIdNeedToDelete = indexes.get(positionNeedToDelete).getSequenceID();
+            return snapshotSegmentsWriterFuture.thenCompose(writer -> writer.deleteAsync(buildKey(sequenceIdNeedToDelete), null))
+                    .thenRun(() -> {
+                        if (log.isDebugEnabled()) {
+                            log.debug("[{}] Successes to delete the snapshot segment, "
+                                            + "whose sequenceId is [{}] and maxReadPosition is [{}]",
+                                    this.topic.getName(), this.sequenceID, positionNeedToDelete);
+                        }
+                        abortTxnSegments.remove(positionNeedToDelete);
+                        //The process will check whether the snapshot segment is null, and update index when recovered.
+                        indexes.remove(positionNeedToDelete);
+                        //Keep index snapshot and update index
+                        updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                indexes.values().stream().toList());
+                    }).exceptionally(e -> {
+                        log.warn("[{}] Failed to delete the snapshot segment, "
+                                        + "whose sequenceId is [{}] and maxReadPosition is [{}]",
+                                this.topic.getName(), this.sequenceID, positionNeedToDelete, e);
+                        return null;
+                    });
+        }
+
+        //Update the indexes with the giving index snapshot and indexlist in the transactionBufferSnapshotIndexe.
+        private CompletableFuture<Void> updateSnapshotIndex(TransactionBufferSnapshotIndexesMetadata snapshotSegment,
+                                                            List<TransactionBufferSnapshotIndex> indexList) {
+            TransactionBufferSnapshotIndexes snapshotIndexes = new TransactionBufferSnapshotIndexes();
+            return snapshotIndexWriterFuture
+                    .thenCompose((indexesWriter) -> {
+                        snapshotIndexes.setIndexList(indexList);
+                        snapshotIndexes.setSnapshot(snapshotSegment);
+                        return indexesWriter.writeAsync(topic.getName(), snapshotIndexes);
+                    })
+                    .thenRun(() -> {
+                        persistentSnapshotIndexes = snapshotIndexes;
+                        lastSnapshotTimestamps = System.currentTimeMillis();
+                    })
+                    .exceptionally(e -> {
+                        log.error("[{}] Failed to update snapshot segment index", snapshotIndexes.getTopicName(), e);
+                        return null;
+                    });
+        }
+
+        //Only update the metadata in the transactionBufferSnapshotIndexes.
+        private CompletableFuture<Void> updateIndexMetadataForTheLastSnapshot(PositionImpl maxReadPosition,
+                                                                              ArrayList<TxnID> abortedTxns) {
+            TransactionBufferSnapshotIndexesMetadata metadata = new TransactionBufferSnapshotIndexesMetadata(
+                    maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(), serializationForSegment(abortedTxns));
+
+            return updateSnapshotIndex(metadata, persistentSnapshotIndexes.getIndexList());
+        }
+
+        protected CompletableFuture<Void> clearSnapshotSegmentAndIndexes() {
+            ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+            //Delete all segment
+            while (!abortTxnSegments.isEmpty()) {
+                if (log.isDebugEnabled()) {
+                    log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                            topic.getName(), abortTxnSegments.firstKey());
+                }
+                PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+                completableFutures.add(persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+            }
+            //Delete index
+            return FutureUtil.waitForAll(completableFutures)
+                    .thenCompose((ignore) -> snapshotIndexWriterFuture
+                            .thenCompose(indexesWriter -> indexesWriter.writeAsync(topic.getName(), null)))
+                    .thenRun(() -> {
+                        log.info("Successes to clear the snapshot segment and indexes for the topic [{}]",
+                                topic.getName());
+
+                    })
+                    .exceptionally(e -> {
+                        log.error("Failed to clear the snapshot segment and indexes for the topic [{}]",
+                                topic.getName(), e);
+
+                        return null;
+                    });
+        }
+
+
+        CompletableFuture<Void> closeAsync() {
+            return CompletableFuture.allOf(
+                    this.snapshotIndexWriterFuture.thenCompose(SystemTopicClient.Writer::closeAsync),
+                    this.snapshotSegmentsWriterFuture.thenCompose(SystemTopicClient.Writer::closeAsync));
+        }
+    }
+
+    ArrayList<TxnID> deserializationFotSnapshotSegment(List<TxnIDData> snapshotSegment) {

Review Comment:
   Why not this method with the modifier `private`?
   
   Same for `serializationForSegment `



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

Review Comment:
   Suggest `whenComplete` instead `thenRun & exceptionally`



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,600 @@
+/**
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final Timer timer;
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            abortTxnSegments.put(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            unsealedAbortedTxnIdSegment = new ArrayList<>();
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition) {
+        return takeAbortedTxnSnapshot(maxReadPosition, unsealedAbortedTxnIdSegment);
+    }
+
+    private CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition, ArrayList<TxnID> aborts) {
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                () -> persistentWorker
+                        .updateIndexMetadataForTheLastSnapshot(maxReadPosition, aborts)
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);

Review Comment:
   We can close `reader` with `try...catch...finally` here.



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


[GitHub] [pulsar] congbobo184 merged pull request #17847: [feat][txn] Implement the AbortedTxnProcessor for TransactionBuffer

Posted by GitBox <gi...@apache.org>.
congbobo184 merged PR #17847:
URL: https://github.com/apache/pulsar/pull/17847


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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008098108


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,672 @@
+/**
+ * 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.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.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        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 -> {
+                                //TODO: read on demand
+                                CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>();
+                                completableFutures.add(handleSegmentFuture);
+                                readOnlyManagedLedger.asyncReadEntry(
+                                        new PositionImpl(index.getPersistentPositionLedgerID(),
+                                                index.getPersistentPositionEntryID()),
+                                        new AsyncCallbacks.ReadEntryCallback() {
+                                            @Override
+                                            public void readEntryComplete(Entry entry, Object ctx) {
+                                                //Remove invalid index
+                                                if (entry == null) {
+                                                    indexes.remove(new PositionImpl(
+                                                            index.getMaxReadPositionLedgerID(),
+                                                            index.getMaxReadPositionEntryID()));
+                                                    handleSegmentFuture.complete(null);
+                                                    invalidIndex.getAndIncrement();
+                                                    return;
+                                                }
+                                                handleSnapshotSegmentEntry(entry);
+                                                handleSegmentFuture.complete(null);
+                                            }
+
+                                            @Override
+                                            public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                                                handleSegmentFuture.completeExceptionally(exception);
+                                            }
+                                        }, null);
+                            });
+                        }
+
+                            @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.
+                        return FutureUtil.waitForAll(completableFutures).thenCompose((ignore) -> {
+                            if (invalidIndex.get() != 0 ) {
+                                persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, ()
+                                        -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(),
+                                        indexes.values().stream().toList()));
+                            }
+                            return CompletableFuture.completedFuture(finalStartReadCursorPosition);
+                        });
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.Close,
+                () -> persistentWorker.clearSnapshotSegmentAndIndexes()
+                        .thenRun(() -> {
+                            completableFuture.thenCompose(null);
+                        }).exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public PositionImpl getMaxReadPosition() {
+        return this.maxReadPosition;
+    }
+
+    @Override
+    public long getLastSnapshotTimestamps() {
+        return this.lastSnapshotTimestamps;
+    }
+
+    @Override
+    public CompletableFuture<Void> closeAsync() {
+        return persistentWorker.closeAsync();
+    }
+
+    private void handleSnapshotSegmentEntry(Entry entry) {
+        //decode snapshot from entry
+        ByteBuf headersAndPayload = entry.getDataBuffer();
+        //skip metadata
+        Commands.parseMessageMetadata(headersAndPayload);
+        TransactionBufferSnapshotSegment snapshotSegment = Schema.AVRO(TransactionBufferSnapshotSegment.class)
+                .decode(Unpooled.wrappedBuffer(headersAndPayload).nioBuffer());
+        abortTxnSegments.put(new PositionImpl(snapshotSegment.getMaxReadPositionLedgerId(),
+                snapshotSegment.getMaxReadPositionEntryId()), deserializationFotSnapshotSegment(
+                snapshotSegment.getAborts()));
+
+    }
+
+    private <T> void  closeReader(SystemTopicClient.Reader<T> reader) {
+        reader.closeAsync().exceptionally(e -> {
+            log.error("[{}]Transaction buffer snapshot reader close error!", topic.getName(), e);
+            return null;
+        });
+    }
+
+    private class PersistentWorker {
+        protected final AtomicLong sequenceID = new AtomicLong(0);
+
+        private final PersistentTopic topic;
+
+        //Persistent snapshot segment and index at the single thread.
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotSegment>>
+                snapshotSegmentsWriterFuture;
+        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshotIndexes>>
+                snapshotIndexWriterFuture;
+
+        private enum OperationState {
+            None,
+            UpdatingIndex,
+            WritingSegment,
+            DeletingSegment,
+            Closing,
+            Closed
+        }
+        private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState>
+                STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class,
+                        PersistentWorker.OperationState.class, "operationState");
+
+        public enum OperationType {
+            UpdateIndex,
+            WriteSegment,
+            DeleteSegment,
+            Close
+        }
+
+        private volatile OperationState operationState = OperationState.None;
+
+        ConcurrentSkipListMap<OperationType, Supplier<CompletableFuture<Void>>> taskQueue =

Review Comment:
   Ok, I change it to ConcurrentLinkedDeque<Pair<OperationType, Supplier<CompletableFuture<Void>>>>.



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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1007592792


##########
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:
   We can not do this, because this system topic is shared by all topics in a namespace



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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1007592306


##########
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:
   I think i is not outdated



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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1007592306


##########
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:
   I think i is not outdated



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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1007890424


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,672 @@
+/**
+ * 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.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.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    //Store the latest aborted transaction IDs and the latest max read position.
+    private PositionImpl maxReadPosition;
+    private ArrayList<TxnID> unsealedAbortedTxnIdSegment = new ArrayList<>();
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, ArrayList<TxnID>> abortTxnSegments
+            = new ConcurrentSkipListMap<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes
+            = new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final 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;
+    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();
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl maxReadPosition) {
+        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(maxReadPosition, unsealedAbortedTxnIdSegment);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(unsealedAbortedTxnIdSegment, maxReadPosition));
+            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(TxnID txnID, Position readPosition) {
+        if (readPosition == null) {
+            return abortTxnSegments.values().stream()
+                    .anyMatch(list -> list.contains(txnID)) || unsealedAbortedTxnIdSegment.contains(txnID);
+        } else {
+            PositionImpl maxReadPosition = abortTxnSegments.ceilingKey((PositionImpl) readPosition);
+            if (maxReadPosition != null) {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo(maxReadPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)));
+            } else {
+                return abortTxnSegments.keySet().stream()
+                        .filter((position) -> position.compareTo((PositionImpl) readPosition) <= 0)
+                        .anyMatch((position -> abortTxnSegments.get(position).contains(txnID)))
+                        || unsealedAbortedTxnIdSegment.contains(txnID);
+            }
+        }
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!abortTxnSegments.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(abortTxnSegments.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), abortTxnSegments.firstKey());
+            }
+            PositionImpl positionNeedToDelete = abortTxnSegments.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return "multiple-" + sequenceId + this.topic.getName();
+    }
+
+    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,
+                () -> persistentWorker.updateIndexMetadataForTheLastSnapshot()
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+                    closeReader(reader);
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = deserializationFotSnapshotSegment(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        this.maxReadPosition = new PositionImpl(persistentSnapshotIndexes
+                                .getSnapshot().getMaxReadPositionLedgerId(),
+                                persistentSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                        if (indexes.size() != 0) {
+                            persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);

Review Comment:
   If index.size() = 0, the sequenceID may not 0, we need to think about how to get the correct sequnceID, may it not the sequenceID, it is segementID is right



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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008072382


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -625,108 +520,76 @@ public static class TopicTransactionBufferRecover implements Runnable {
 
         private final TopicTransactionBuffer topicTransactionBuffer;
 
-        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+        private final AbortedTxnProcessor abortedTxnProcessor;
 
         private TopicTransactionBufferRecover(TopicTransactionBufferRecoverCallBack callBack, PersistentTopic topic,
-                                              TopicTransactionBuffer transactionBuffer, CompletableFuture<
-                SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter) {
+                                              TopicTransactionBuffer transactionBuffer,
+                                              AbortedTxnProcessor abortedTxnProcessor) {
             this.topic = topic;
             this.callBack = callBack;
             this.entryQueue = new SpscArrayQueue<>(2000);
             this.topicTransactionBuffer = transactionBuffer;
-            this.takeSnapshotWriter = takeSnapshotWriter;
+            this.abortedTxnProcessor = abortedTxnProcessor;
         }
 
         @SneakyThrows
         @Override
         public void run() {
-            this.takeSnapshotWriter.thenRunAsync(() -> {
-                if (!this.topicTransactionBuffer.changeToInitializingState()) {
-                    log.warn("TransactionBuffer {} of topic {} can not change state to Initializing",
-                            this, topic.getName());
+            if (!this.topicTransactionBuffer.changeToInitializingState()) {
+                log.warn("TransactionBuffer {} of topic {} can not change state to Initializing",
+                        this, topic.getName());
+                return;
+            }
+            abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> {
+                //Transaction is not enable for this topic, so just make maxReadPosition as LAC.

Review Comment:
   Yes, the logic of recovery is actually that.
   ```
       abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> {
           //Transaction is not enable for this topic, so just make maxReadPosition as LAC.
           if (startReadCursorPosition == null) {
               callBack.noNeedToRecover();
               return;
           } else {
   
   ```
   



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


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

Posted by GitBox <gi...@apache.org>.
liangyepianzhou commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008072382


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java:
##########
@@ -625,108 +520,76 @@ public static class TopicTransactionBufferRecover implements Runnable {
 
         private final TopicTransactionBuffer topicTransactionBuffer;
 
-        private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+        private final AbortedTxnProcessor abortedTxnProcessor;
 
         private TopicTransactionBufferRecover(TopicTransactionBufferRecoverCallBack callBack, PersistentTopic topic,
-                                              TopicTransactionBuffer transactionBuffer, CompletableFuture<
-                SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter) {
+                                              TopicTransactionBuffer transactionBuffer,
+                                              AbortedTxnProcessor abortedTxnProcessor) {
             this.topic = topic;
             this.callBack = callBack;
             this.entryQueue = new SpscArrayQueue<>(2000);
             this.topicTransactionBuffer = transactionBuffer;
-            this.takeSnapshotWriter = takeSnapshotWriter;
+            this.abortedTxnProcessor = abortedTxnProcessor;
         }
 
         @SneakyThrows
         @Override
         public void run() {
-            this.takeSnapshotWriter.thenRunAsync(() -> {
-                if (!this.topicTransactionBuffer.changeToInitializingState()) {
-                    log.warn("TransactionBuffer {} of topic {} can not change state to Initializing",
-                            this, topic.getName());
+            if (!this.topicTransactionBuffer.changeToInitializingState()) {
+                log.warn("TransactionBuffer {} of topic {} can not change state to Initializing",
+                        this, topic.getName());
+                return;
+            }
+            abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> {
+                //Transaction is not enable for this topic, so just make maxReadPosition as LAC.

Review Comment:
   Yes, the logic of recovery is actually that. What is wrong?
   ```
       abortedTxnProcessor.recoverFromSnapshot().thenAcceptAsync(startReadCursorPosition -> {
           //Transaction is not enable for this topic, so just make maxReadPosition as LAC.
           if (startReadCursorPosition == null) {
               callBack.noNeedToRecover();
               return;
           } else {
   
   ```
   



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


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

Posted by GitBox <gi...@apache.org>.
congbobo184 commented on code in PR #17847:
URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008651764


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,255 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position maxReadPosition) {
+        if (this.maxReadPosition != maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) maxReadPosition;
+            takeSnapshotByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                aborts.firstKey();

Review Comment:
   delete



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,255 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();

Review Comment:
   the same as line 69



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,255 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position maxReadPosition) {
+        if (this.maxReadPosition != maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) maxReadPosition;
+            takeSnapshotByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                aborts.firstKey();
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    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;
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasSnapshot) {
+                            return CompletableFuture.completedFuture(null);
+                        }
+                        return CompletableFuture.completedFuture(startReadCursorPosition);

Review Comment:
   return `startReadCursorPosition `, dont need hasSnapshot



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,255 @@
+/**
+ * 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.util.Timeout;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.commons.collections4.map.LinkedMap;
+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.AbortTxnMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+    private final PersistentTopic topic;
+    private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter;
+    private volatile PositionImpl maxReadPosition;
+
+    private final Timer timer;
+
+    /**
+     * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this
+     * position have been deleted.
+     */
+    private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>();
+
+    private volatile long lastSnapshotTimestamps;
+    private final int takeSnapshotIntervalNumber;
+
+    private final int takeSnapshotIntervalTime;
+
+
+    // 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();
+
+
+    public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+        this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar()
+                .getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName()));
+        this.timer = topic.getBrokerService().getPulsar().getTransactionTimer();
+        this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount();
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
+    }
+
+    @Override
+    public void appendAbortedTxn(TxnID abortedTxnId, PositionImpl position) {
+        aborts.put(abortedTxnId, position);
+    }
+
+    @Override
+    public void updateMaxReadPosition(Position maxReadPosition) {
+        if (this.maxReadPosition != maxReadPosition) {
+            this.maxReadPosition = (PositionImpl) maxReadPosition;
+            takeSnapshotByChangeTimes();
+        }
+    }
+
+    @Override
+    public void updateMaxReadPositionNotIncreaseChangeTimes(Position maxReadPosition) {
+        this.maxReadPosition = (PositionImpl) maxReadPosition;
+    }
+
+    @Override
+    public void trimExpiredAbortedTxns() {
+        while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                aborts.firstKey();
+                log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}",
+                        topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey()));
+            }
+            aborts.remove(aborts.firstKey());
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.containsKey(txnID);
+    }
+
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    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;
+                                    handleSnapshot(transactionBufferSnapshot);
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshot.getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshot.getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                        closeReader(reader);
+                        if (!hasSnapshot) {
+                            return CompletableFuture.completedFuture(null);
+                        }
+                        return CompletableFuture.completedFuture(startReadCursorPosition);
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        closeReader(reader);
+                        return FutureUtil.failedFuture(ex);
+                    }
+
+                },  topic.getBrokerService().getPulsar().getTransactionExecutorProvider()
+                        .getExecutor(this));
+    }
+
+    @Override
+    public CompletableFuture<Void> clearAndCloseAsync() {
+        timer.stop();
+        return this.takeSnapshotWriter.thenCompose(writer -> {
+            TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot();
+            snapshot.setTopicName(topic.getName());
+            return writer.deleteAsync(snapshot.getTopicName(), snapshot);
+        }).thenCompose(__ -> CompletableFuture.completedFuture(null));

Review Comment:
   need close writer



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