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/12/28 17:03:57 UTC

[GitHub] [pulsar] poorbarcode commented on a diff in pull request #18273: [feat][txn] implement the SnapshotSegmentAbortedTxnProcessor

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


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,596 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,2
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private LinkedList<TxnID> unsealedAbortedTxnIdSegment;
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, TxnID> segmentIndex = new ConcurrentSkipListMap<>();
+
+    private final LinkedList<TxnID> aborts = new LinkedList<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes =
+            new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    private static final String SNAPSHOT_PREFIX = "multiple-";
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.unsealedAbortedTxnIdSegment = new LinkedList<>();
+    }
+
+    @Override
+    public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl abortedMarkerPersistentPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        aborts.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            LinkedList<TxnID> abortedSegment = unsealedAbortedTxnIdSegment;
+            segmentIndex.put(abortedMarkerPersistentPosition, abortedTxnId);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(abortedSegment, abortedMarkerPersistentPosition));
+            this.unsealedAbortedTxnIdSegment = new LinkedList<>();
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.contains(txnID);
+    }
+
+    //In this implementation, we adopt snapshot segments. And then we clear invalid segment by its max read position.
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!segmentIndex.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(segmentIndex.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), segmentIndex.firstKey());
+            }
+            PositionImpl positionNeedToDelete = segmentIndex.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return SNAPSHOT_PREFIX + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnsSnapshot(PositionImpl maxReadPosition) {
+        TransactionBufferSnapshotIndexesMetadata metadata = new TransactionBufferSnapshotIndexesMetadata(
+                maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(),
+                convertTypeToTxnIDData(unsealedAbortedTxnIdSegment));
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                () -> persistentWorker
+                        .updateSnapshotIndex(metadata, persistentSnapshotIndexes.getIndexList())
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();

Review Comment:
   Let's instead `reader.readNext()` to `reader.readNextAsync().get()` and catch `timeout ex`, just like https://github.com/apache/pulsar/pull/18833



##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java:
##########
@@ -0,0 +1,596 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,2
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.transaction.buffer.impl;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.Timer;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.mledger.AsyncCallbacks;
+import org.apache.bookkeeper.mledger.Entry;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
+import org.apache.bookkeeper.mledger.Position;
+import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
+import org.apache.bookkeeper.mledger.impl.PositionImpl;
+import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.pulsar.broker.service.persistent.PersistentTopic;
+import org.apache.pulsar.broker.systopic.SystemTopicClient;
+import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndex;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexes;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotIndexesMetadata;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TransactionBufferSnapshotSegment;
+import org.apache.pulsar.broker.transaction.buffer.metadata.v2.TxnIDData;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.transaction.TxnID;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+import org.apache.pulsar.common.events.EventType;
+import org.apache.pulsar.common.naming.TopicDomain;
+import org.apache.pulsar.common.naming.TopicName;
+import org.apache.pulsar.common.protocol.Commands;
+import org.apache.pulsar.common.util.FutureUtil;
+
+@Slf4j
+public class SnapshotSegmentAbortedTxnProcessorImpl implements AbortedTxnProcessor {
+
+    private LinkedList<TxnID> unsealedAbortedTxnIdSegment;
+
+    //Store the fixed aborted transaction segment
+    private final ConcurrentSkipListMap<PositionImpl, TxnID> segmentIndex = new ConcurrentSkipListMap<>();
+
+    private final LinkedList<TxnID> aborts = new LinkedList<>();
+
+    private final ConcurrentSkipListMap<PositionImpl, TransactionBufferSnapshotIndex> indexes =
+            new ConcurrentSkipListMap<>();
+    //The latest persistent snapshot index. This is used to combine new segment indexes with the latest metadata and
+    // indexes.
+    private TransactionBufferSnapshotIndexes persistentSnapshotIndexes = new TransactionBufferSnapshotIndexes();
+
+    private final PersistentTopic topic;
+
+    private volatile long lastSnapshotTimestamps;
+
+    private final int takeSnapshotIntervalTime;
+
+    private final int transactionBufferMaxAbortedTxnsOfSnapshotSegment;
+    private final PersistentWorker persistentWorker;
+
+    private static final String SNAPSHOT_PREFIX = "multiple-";
+
+    public SnapshotSegmentAbortedTxnProcessorImpl(PersistentTopic topic) {
+        this.topic = topic;
+        this.persistentWorker = new PersistentWorker(topic);
+        this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis();
+        this.transactionBufferMaxAbortedTxnsOfSnapshotSegment =  topic.getBrokerService().getPulsar()
+                .getConfiguration().getTransactionBufferSnapshotSegmentSize();
+        this.unsealedAbortedTxnIdSegment = new LinkedList<>();
+    }
+
+    @Override
+    public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl abortedMarkerPersistentPosition) {
+        unsealedAbortedTxnIdSegment.add(abortedTxnId);
+        aborts.add(abortedTxnId);
+        //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment.
+        if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) {
+            LinkedList<TxnID> abortedSegment = unsealedAbortedTxnIdSegment;
+            segmentIndex.put(abortedMarkerPersistentPosition, abortedTxnId);
+            persistentWorker.appendTask(PersistentWorker.OperationType.WriteSegment, () ->
+                    persistentWorker.takeSnapshotSegmentAsync(abortedSegment, abortedMarkerPersistentPosition));
+            this.unsealedAbortedTxnIdSegment = new LinkedList<>();
+        }
+    }
+
+    @Override
+    public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) {
+        return aborts.contains(txnID);
+    }
+
+    //In this implementation, we adopt snapshot segments. And then we clear invalid segment by its max read position.
+    @Override
+    public void trimExpiredAbortedTxns() {
+        //Checking whether there are some segment expired.
+        while (!segmentIndex.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger())
+                .ledgerExists(segmentIndex.firstKey().getLedgerId())) {
+            if (log.isDebugEnabled()) {
+                log.debug("[{}] Topic transaction buffer clear aborted transactions, maxReadPosition : {}",
+                        topic.getName(), segmentIndex.firstKey());
+            }
+            PositionImpl positionNeedToDelete = segmentIndex.firstKey();
+            persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment,
+                    () -> persistentWorker.deleteSnapshotSegment(positionNeedToDelete));
+        }
+    }
+
+    private String buildKey(long sequenceId) {
+        return SNAPSHOT_PREFIX + sequenceId + "-" + this.topic.getName();
+    }
+
+    @Override
+    public CompletableFuture<Void> takeAbortedTxnsSnapshot(PositionImpl maxReadPosition) {
+        TransactionBufferSnapshotIndexesMetadata metadata = new TransactionBufferSnapshotIndexesMetadata(
+                maxReadPosition.getLedgerId(), maxReadPosition.getEntryId(),
+                convertTypeToTxnIDData(unsealedAbortedTxnIdSegment));
+        CompletableFuture<Void> completableFuture = new CompletableFuture<>();
+        persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex,
+                () -> persistentWorker
+                        .updateSnapshotIndex(metadata, persistentSnapshotIndexes.getIndexList())
+                        .thenRun(() -> completableFuture.complete(null))
+                        .exceptionally(e -> {
+                            completableFuture.completeExceptionally(e);
+                            return null;
+                        }));
+        return completableFuture;
+    }
+
+    @Override
+    public CompletableFuture<PositionImpl> recoverFromSnapshot() {
+        return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory()
+                .getTxnBufferSnapshotIndexService()
+                .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> {
+                    PositionImpl startReadCursorPosition = null;
+                    boolean hasIndex = false;
+                    try {
+                        //Read Index to recover the sequenceID, indexes, lastAbortedTxns and maxReadPosition.
+                        while (reader.hasMoreEvents()) {
+                            Message<TransactionBufferSnapshotIndexes> message = reader.readNext();
+                            if (topic.getName().equals(message.getKey())) {
+                                TransactionBufferSnapshotIndexes transactionBufferSnapshotIndexes = message.getValue();
+                                if (transactionBufferSnapshotIndexes != null) {
+                                    hasIndex = true;
+                                    this.persistentSnapshotIndexes = transactionBufferSnapshotIndexes;
+                                    startReadCursorPosition = PositionImpl.get(
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionLedgerId(),
+                                            transactionBufferSnapshotIndexes.getSnapshot().getMaxReadPositionEntryId());
+                                }
+                            }
+                        }
+                    } catch (Exception ex) {
+                        log.error("[{}] Transaction buffer recover fail when read "
+                                + "transactionBufferSnapshot!", topic.getName(), ex);
+                        return FutureUtil.failedFuture(ex);
+                    } finally {
+                        closeReader(reader);
+                    }
+                    PositionImpl finalStartReadCursorPosition = startReadCursorPosition;
+                    if (!hasIndex) {
+                        return CompletableFuture.completedFuture(null);
+                    } else {
+                        persistentSnapshotIndexes.getIndexList()
+                                .forEach(transactionBufferSnapshotIndex ->
+                                        indexes.put(new PositionImpl(
+                                                        transactionBufferSnapshotIndex.persistentPositionLedgerID,
+                                                        transactionBufferSnapshotIndex.persistentPositionEntryID),
+                                                transactionBufferSnapshotIndex));
+                        this.unsealedAbortedTxnIdSegment = convertTypeToTxnID(persistentSnapshotIndexes
+                                .getSnapshot().getAborts());
+                        if (indexes.size() != 0) {
+                            persistentWorker.sequenceID.set(indexes.lastEntry().getValue().sequenceID + 1);
+                        }
+                    }
+                    //Read snapshot segment to recover aborts.
+                    ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>();
+                    CompletableFuture<Void> openManagedLedgerFuture = new CompletableFuture<>();
+                    AtomicLong invalidIndex = new AtomicLong(0);

Review Comment:
   I see that the variable `invalidIndex` is only used to identify whether the segment is valid or not, so why not use `boolean`



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