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

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

Reply via email to