congbobo184 commented on code in PR #18273: URL: https://github.com/apache/pulsar/pull/18273#discussion_r1067649532
########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java: ########## @@ -0,0 +1,651 @@ +/** + * 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 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.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Function; +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.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; +import org.apache.commons.collections4.map.LinkedMap; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.service.BrokerServiceException; +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.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.SystemTopicNames; +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 { + + //Stored the unsealed aborted txn id, it will be persistent as a snapshot segment and reinit + // when its size reach the capital of a snapshot segment. + private LinkedList<TxnID> unsealedAbortedTxnIdSegment; + + //A mapping form the latest txn mark persistent position in a segment to its latest txn ID. + //This is mainly used to trim expired snapshot segment and remove them form aborts. + private final LinkedMap<PositionImpl, TxnID> segmentIndex = new LinkedMap<>(); + + //Store all aborted txn IDs check whether a txn is an aborted txn. + private final LinkedMap<TxnID, TxnID> aborts = new LinkedMap<>(); + //The indexes of the snapshot segments whose key is the aborted mark persistent position. + private final LinkedMap<PositionImpl, TransactionBufferSnapshotIndex> indexes = new LinkedMap<>(); + //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 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.transactionBufferMaxAbortedTxnsOfSnapshotSegment = topic.getBrokerService().getPulsar() + .getConfiguration().getTransactionBufferSnapshotSegmentSize(); + this.unsealedAbortedTxnIdSegment = new LinkedList<>(); + } + + @Override + public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl abortedMarkerPersistentPosition) { + unsealedAbortedTxnIdSegment.add(abortedTxnId); + aborts.put(abortedTxnId, abortedTxnId); + //The size of lastAbortedTxns reaches the configuration of the size of snapshot segment. + if (unsealedAbortedTxnIdSegment.size() == transactionBufferMaxAbortedTxnsOfSnapshotSegment) { Review Comment: transactionBufferMaxAbortedTxnsOfSnapshotSegment is memory size, not number size ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java: ########## @@ -0,0 +1,651 @@ +/** + * 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 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.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Function; +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.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; +import org.apache.commons.collections4.map.LinkedMap; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.service.BrokerServiceException; +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.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.SystemTopicNames; +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 { + + //Stored the unsealed aborted txn id, it will be persistent as a snapshot segment and reinit + // when its size reach the capital of a snapshot segment. + private LinkedList<TxnID> unsealedAbortedTxnIdSegment; + + //A mapping form the latest txn mark persistent position in a segment to its latest txn ID. + //This is mainly used to trim expired snapshot segment and remove them form aborts. + private final LinkedMap<PositionImpl, TxnID> segmentIndex = new LinkedMap<>(); + + //Store all aborted txn IDs check whether a txn is an aborted txn. + private final LinkedMap<TxnID, TxnID> aborts = new LinkedMap<>(); + //The indexes of the snapshot segments whose key is the aborted mark persistent position. + private final LinkedMap<PositionImpl, TransactionBufferSnapshotIndex> indexes = new LinkedMap<>(); + //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 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.transactionBufferMaxAbortedTxnsOfSnapshotSegment = topic.getBrokerService().getPulsar() + .getConfiguration().getTransactionBufferSnapshotSegmentSize(); + this.unsealedAbortedTxnIdSegment = new LinkedList<>(); + } + + @Override + public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl abortedMarkerPersistentPosition) { + unsealedAbortedTxnIdSegment.add(abortedTxnId); + aborts.put(abortedTxnId, 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, (cancel) -> + persistentWorker.takeSnapshotSegmentAsync(abortedSegment, abortedMarkerPersistentPosition)); + this.unsealedAbortedTxnIdSegment = new LinkedList<>(); + } + } + + @Override + public boolean checkAbortedTransaction(TxnID txnID) { + return aborts.containsKey(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. + List<PositionImpl> positionsNeedToDelete = new ArrayList<>(); + 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(); + positionsNeedToDelete.add(positionNeedToDelete); + + TxnID theLatestDeletedTxnID = segmentIndex.remove(positionNeedToDelete); + //The position is already deleted. + if (theLatestDeletedTxnID == null) { + continue; + } Review Comment: what time the theLatestDeletedTxnID can be null? ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java: ########## @@ -0,0 +1,651 @@ +/** + * 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 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.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Function; +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.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; +import org.apache.commons.collections4.map.LinkedMap; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.service.BrokerServiceException; +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.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.SystemTopicNames; +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 { + + //Stored the unsealed aborted txn id, it will be persistent as a snapshot segment and reinit + // when its size reach the capital of a snapshot segment. + private LinkedList<TxnID> unsealedAbortedTxnIdSegment; + + //A mapping form the latest txn mark persistent position in a segment to its latest txn ID. + //This is mainly used to trim expired snapshot segment and remove them form aborts. + private final LinkedMap<PositionImpl, TxnID> segmentIndex = new LinkedMap<>(); + + //Store all aborted txn IDs check whether a txn is an aborted txn. + private final LinkedMap<TxnID, TxnID> aborts = new LinkedMap<>(); + //The indexes of the snapshot segments whose key is the aborted mark persistent position. + private final LinkedMap<PositionImpl, TransactionBufferSnapshotIndex> indexes = new LinkedMap<>(); + //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 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.transactionBufferMaxAbortedTxnsOfSnapshotSegment = topic.getBrokerService().getPulsar() + .getConfiguration().getTransactionBufferSnapshotSegmentSize(); + this.unsealedAbortedTxnIdSegment = new LinkedList<>(); + } + + @Override + public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl abortedMarkerPersistentPosition) { + unsealedAbortedTxnIdSegment.add(abortedTxnId); + aborts.put(abortedTxnId, 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, (cancel) -> + persistentWorker.takeSnapshotSegmentAsync(abortedSegment, abortedMarkerPersistentPosition)); + this.unsealedAbortedTxnIdSegment = new LinkedList<>(); + } + } + + @Override + public boolean checkAbortedTransaction(TxnID txnID) { + return aborts.containsKey(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. + List<PositionImpl> positionsNeedToDelete = new ArrayList<>(); + 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(); + positionsNeedToDelete.add(positionNeedToDelete); + + TxnID theLatestDeletedTxnID = segmentIndex.remove(positionNeedToDelete); + //The position is already deleted. + if (theLatestDeletedTxnID == null) { + continue; + } + while (!aborts.firstKey().equals(theLatestDeletedTxnID)) { + aborts.remove(aborts.firstKey()); + } + aborts.remove(theLatestDeletedTxnID); + } + //Batch delete the expired segment and then update segment index. + if (!positionsNeedToDelete.isEmpty()) { + persistentWorker.appendTask(PersistentWorker.OperationType.DeleteSegment, + (cancel) -> persistentWorker.deleteSnapshotSegment(positionsNeedToDelete) + .thenRun(() -> persistentWorker.updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(), + indexes.values().stream().toList()))); + } + } + + 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)); + return updateSnapshotIndexMetadata(metadata); + } + + private CompletableFuture<Void> updateSnapshotIndexMetadata(TransactionBufferSnapshotIndexesMetadata metadata) { + CompletableFuture<Void> completableFuture = new CompletableFuture<>(); + persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, (cancel) -> { + if (!cancel) { + return persistentWorker + .updateSnapshotIndex(metadata, persistentSnapshotIndexes.getIndexList()) + .thenRun(() -> completableFuture.complete(null)) + .exceptionally(e -> { + completableFuture.completeExceptionally(e); + return null; + }); + } else { + completableFuture.complete(null); + return completableFuture; + } + }); + 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.readNextAsync() + .get(getSystemClientOperationTimeoutMs(), TimeUnit.MILLISECONDS); + 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 (TimeoutException ex) { + Throwable t = FutureUtil.unwrapCompletionException(ex); + String errorMessage = String.format("[%s] Transaction buffer recover fail by read " + + "transactionBufferSnapshot timeout!", topic.getName()); + log.error(errorMessage, t); + return FutureUtil.failedFuture( + new BrokerServiceException.ServiceUnitNotReadyException(errorMessage, t)); + } 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.abortedMarkLedgerID, + transactionBufferSnapshotIndex.abortedMarkEntryID), + transactionBufferSnapshotIndex)); + this.unsealedAbortedTxnIdSegment = convertTypeToTxnID(persistentSnapshotIndexes + .getSnapshot().getAborts()); + //If the size of indexes is 0, the sequence ID will be the init value 0. + if (indexes.size() != 0) { + persistentWorker.sequenceID.set(indexes.get(indexes.lastKey()).sequenceID + 1); + } + } + //Read snapshot segment to recover aborts. + ArrayList<CompletableFuture<Void>> completableFutures = new ArrayList<>(); + CompletableFuture<Void> openManagedLedgerFuture = new CompletableFuture<>(); + AtomicBoolean hasInvalidIndex = new AtomicBoolean(false); + AsyncCallbacks.OpenReadOnlyManagedLedgerCallback callback = new AsyncCallbacks + .OpenReadOnlyManagedLedgerCallback() { + @Override + public void openReadOnlyManagedLedgerComplete(ReadOnlyManagedLedgerImpl readOnlyManagedLedger, + Object ctx) { + persistentSnapshotIndexes.getIndexList().forEach(index -> { + CompletableFuture<Void> handleSegmentFuture = new CompletableFuture<>(); + completableFutures.add(handleSegmentFuture); + readOnlyManagedLedger.asyncReadEntry( + new PositionImpl(index.getSegmentLedgerID(), + index.getSegmentEntryID()), + new AsyncCallbacks.ReadEntryCallback() { + @Override + public void readEntryComplete(Entry entry, Object ctx) { + handleSnapshotSegmentEntry(entry); + entry.release(); + handleSegmentFuture.complete(null); + } + + @Override + public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + if (exception instanceof ManagedLedgerException + .NonRecoverableLedgerException) { + if (((ManagedLedgerImpl)topic.getManagedLedger()) + .ledgerExists(index.getAbortedMarkLedgerID())) { + log.error("[{}] Failed to read snapshot segment [{}:{}]", + topic.getName(), index.segmentLedgerID, + index.segmentEntryID, exception); + topic.close(); + handleSegmentFuture.completeExceptionally(exception); + } else { + indexes.remove(new PositionImpl( + index.getAbortedMarkLedgerID(), + index.getAbortedMarkEntryID())); + hasInvalidIndex.set(true); + } + } + } + }, null); + }); + openManagedLedgerFuture.complete(null); + } + + @Override + public void openReadOnlyManagedLedgerFailed(ManagedLedgerException exception, Object ctx) { + log.error("[{}] Failed to open readOnly managed ledger", topic, exception); + openManagedLedgerFuture.completeExceptionally(exception); + } + }; + + TopicName snapshotIndexTopicName = TopicName.get(TopicDomain.persistent.toString(), + TopicName.get(topic.getName()).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS); + 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 openManagedLedgerFuture + .thenCompose((ignore) -> FutureUtil.waitForAll(completableFutures).thenCompose((i) -> { + //This is a compensation mechanism for deleting the segment + // but not successfully updating the index. + if (hasInvalidIndex.get()) { + persistentWorker.appendTask(PersistentWorker.OperationType.UpdateIndex, (cancel) + -> persistentWorker + .updateSnapshotIndex(persistentSnapshotIndexes.getSnapshot(), + indexes.values().stream().toList())); + } + //Append the aborted txn IDs in the index metadata + //can keep the order of the aborted txn in the aborts. + //So that we can trim the expired snapshot segment in aborts + // according to the latest txn IDs + convertTypeToTxnID(persistentSnapshotIndexes.getSnapshot().getAborts()) + .forEach(txnID -> aborts.put(txnID, txnID)); + return CompletableFuture.completedFuture(finalStartReadCursorPosition); + })).exceptionally(ex -> { + log.error("[{}] Failed to recover snapshot segment", this.topic.getName(), ex); + return null; + }); + + }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider() + .getExecutor(this)); + } + + @Override + public CompletableFuture<Void> clearAbortedTxnSnapshot() { + CompletableFuture<Void> completableFuture = new CompletableFuture<>(); + persistentWorker.appendTask(PersistentWorker.OperationType.Clear, + (cancel) -> 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()); + + TxnIDData lastTxn = snapshotSegment.getAborts().get(snapshotSegment.getAborts().size() - 1); + segmentIndex.put(new PositionImpl(snapshotSegment.getPersistentPositionLedgerId(), + snapshotSegment.getPersistentPositionEntryId()), + new TxnID(lastTxn.getMostSigBits(), lastTxn.getLeastSigBits())); + convertTypeToTxnID(snapshotSegment.getAborts()).forEach(txnID -> aborts.put(txnID, txnID)); + } + + private long getSystemClientOperationTimeoutMs() throws Exception { + PulsarClientImpl pulsarClient = (PulsarClientImpl) topic.getBrokerService().getPulsar().getClient(); + return pulsarClient.getConfiguration().getOperationTimeoutMs(); + } + + 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; + }); + } + + public 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, + Operating, + Closed + } + private static final AtomicReferenceFieldUpdater<PersistentWorker, PersistentWorker.OperationState> + STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentWorker.class, + PersistentWorker.OperationState.class, "operationState"); + + public enum OperationType { + UpdateIndex, + WriteSegment, + DeleteSegment, + Clear + } + + private volatile OperationState operationState = OperationState.None; + + ConcurrentLinkedDeque<Pair<OperationType, Function<Boolean, CompletableFuture<Void>>>> taskQueue = + new ConcurrentLinkedDeque<>(); + private CompletableFuture<Void> lastOperationFuture; + + public PersistentWorker(PersistentTopic topic) { + this.topic = topic; + this.snapshotSegmentsWriterFuture = this.topic.getBrokerService().getPulsar() + .getTransactionBufferSnapshotServiceFactory() Review Comment: snapshotSegmentsWriterFuture create fail also should close topic ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SnapshotSegmentAbortedTxnProcessorImpl.java: ########## @@ -0,0 +1,651 @@ +/** + * 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 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.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Function; +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.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ReadOnlyManagedLedgerImpl; +import org.apache.commons.collections4.map.LinkedMap; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.service.BrokerServiceException; +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.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.SystemTopicNames; +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 { + + //Stored the unsealed aborted txn id, it will be persistent as a snapshot segment and reinit + // when its size reach the capital of a snapshot segment. + private LinkedList<TxnID> unsealedAbortedTxnIdSegment; + + //A mapping form the latest txn mark persistent position in a segment to its latest txn ID. + //This is mainly used to trim expired snapshot segment and remove them form aborts. + private final LinkedMap<PositionImpl, TxnID> segmentIndex = new LinkedMap<>(); + + //Store all aborted txn IDs check whether a txn is an aborted txn. + private final LinkedMap<TxnID, TxnID> aborts = new LinkedMap<>(); + //The indexes of the snapshot segments whose key is the aborted mark persistent position. + private final LinkedMap<PositionImpl, TransactionBufferSnapshotIndex> indexes = new LinkedMap<>(); + //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 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.transactionBufferMaxAbortedTxnsOfSnapshotSegment = topic.getBrokerService().getPulsar() Review Comment: codestyle -- 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]
