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

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

Reply via email to