congbobo184 commented on code in PR #17847: URL: https://github.com/apache/pulsar/pull/17847#discussion_r1008984864
########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java: ########## @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.transaction.buffer; + +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.client.api.transaction.TxnID; + + +public interface AbortedTxnProcessor { + + /** + * After the transaction buffer writes a transaction aborted marker to the topic, + * the transaction buffer will put the aborted txnID and the aborted marker position to AbortedTxnProcessor. + * @param txnID aborted transaction ID. + * @param position the position of the abort txnID + */ + void putAbortedTxnAndPosition(TxnID txnID, PositionImpl position); + + /** + * Clean up invalid aborted transactions. + */ + void trimExpiredAbortedTxns(); + + /** + * Check whether the transaction ID is an aborted transaction ID. + * @param txnID the transaction ID that needs to be checked. + * @param readPosition the read position of the transaction message, can be used to find the segment. + * @return a boolean, whether the transaction ID is an aborted transaction ID. + */ + boolean checkAbortedTransaction(TxnID txnID, Position readPosition); + + /** + * Recover transaction buffer by transaction buffer snapshot. + * @return a Position (startReadCursorPosition) determiner where to start to recover in the original topic. + */ + + CompletableFuture<PositionImpl> recoverFromSnapshot(); + + /** + * Delete the transaction buffer aborted transaction snapshot. + * @return a completableFuture. + */ + CompletableFuture<Void> deleteAbortedTxnSnapshot(); + + /** + * Take the frist snapshot if the topic has no snapshot before. Review Comment: Take aborted transactions snapshot. ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java: ########## @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.transaction.buffer.impl; + +import io.netty.util.Timer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.systopic.SystemTopicClient; +import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; +import org.apache.pulsar.broker.transaction.buffer.metadata.AbortTxnMetadata; +import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; + +@Slf4j +public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor { + private final PersistentTopic topic; + private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter; + /** + * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this + * position have been deleted. + */ + private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>(); + + private volatile long lastSnapshotTimestamps; + private final int takeSnapshotIntervalNumber; + + private final int takeSnapshotIntervalTime; + + public SingleSnapshotAbortedTxnProcessorImpl(PersistentTopic topic) { + this.topic = topic; + this.takeSnapshotWriter = this.topic.getBrokerService().getPulsar() + .getTransactionBufferSnapshotServiceFactory() + .getTxnBufferSnapshotService().createWriter(TopicName.get(topic.getName())); + this.takeSnapshotIntervalNumber = topic.getBrokerService().getPulsar() + .getConfiguration().getTransactionBufferSnapshotMaxTransactionCount(); + this.takeSnapshotIntervalTime = topic.getBrokerService().getPulsar() + .getConfiguration().getTransactionBufferSnapshotMinTimeInMillis(); + } + + @Override + public void putAbortedTxnAndPosition(TxnID abortedTxnId, PositionImpl position) { + aborts.put(abortedTxnId, position); + } + + //In this implementation we clear the invalid aborted txn ID one by one. + @Override + public void trimExpiredAbortedTxns() { + while (!aborts.isEmpty() && !((ManagedLedgerImpl) topic.getManagedLedger()) + .ledgerExists(aborts.get(aborts.firstKey()).getLedgerId())) { + if (log.isDebugEnabled()) { + log.debug("[{}] Topic transaction buffer clear aborted transaction, TxnId : {}, Position : {}", + topic.getName(), aborts.firstKey(), aborts.get(aborts.firstKey())); + } + aborts.remove(aborts.firstKey()); + } + } + + @Override + public boolean checkAbortedTransaction(TxnID txnID, Position readPosition) { + return aborts.containsKey(txnID); + } + + + @Override + public CompletableFuture<PositionImpl> recoverFromSnapshot() { + return topic.getBrokerService().getPulsar().getTransactionBufferSnapshotServiceFactory() + .getTxnBufferSnapshotService() + .createReader(TopicName.get(topic.getName())).thenComposeAsync(reader -> { + PositionImpl startReadCursorPosition = null; + try { + while (reader.hasMoreEvents()) { + Message<TransactionBufferSnapshot> message = reader.readNext(); + if (topic.getName().equals(message.getKey())) { + TransactionBufferSnapshot transactionBufferSnapshot = message.getValue(); + if (transactionBufferSnapshot != null) { + handleSnapshot(transactionBufferSnapshot); + startReadCursorPosition = PositionImpl.get( + transactionBufferSnapshot.getMaxReadPositionLedgerId(), + transactionBufferSnapshot.getMaxReadPositionEntryId()); + } + } + } + closeReader(reader); + return CompletableFuture.completedFuture(startReadCursorPosition); + } catch (Exception ex) { + log.error("[{}] Transaction buffer recover fail when read " + + "transactionBufferSnapshot!", topic.getName(), ex); + closeReader(reader); + return FutureUtil.failedFuture(ex); + } + + }, topic.getBrokerService().getPulsar().getTransactionExecutorProvider() + .getExecutor(this)); + } + + @Override + public CompletableFuture<Void> deleteAbortedTxnSnapshot() { + return this.takeSnapshotWriter.thenCompose(writer -> { + TransactionBufferSnapshot snapshot = new TransactionBufferSnapshot(); + snapshot.setTopicName(topic.getName()); + return writer.deleteAsync(snapshot.getTopicName(), snapshot); + }).thenRun(this::closeAsync); Review Comment: this method don't need close ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/SingleSnapshotAbortedTxnProcessorImpl.java: ########## @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.transaction.buffer.impl; + +import io.netty.util.Timer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.systopic.SystemTopicClient; +import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; +import org.apache.pulsar.broker.transaction.buffer.metadata.AbortTxnMetadata; +import org.apache.pulsar.broker.transaction.buffer.metadata.TransactionBufferSnapshot; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; + +@Slf4j +public class SingleSnapshotAbortedTxnProcessorImpl implements AbortedTxnProcessor { + private final PersistentTopic topic; + private final CompletableFuture<SystemTopicClient.Writer<TransactionBufferSnapshot>> takeSnapshotWriter; + /** + * Aborts, map for jude message is aborted, linked for remove abort txn in memory when this + * position have been deleted. + */ + private final LinkedMap<TxnID, PositionImpl> aborts = new LinkedMap<>(); + + private volatile long lastSnapshotTimestamps; + private final int takeSnapshotIntervalNumber; + + private final int takeSnapshotIntervalTime; Review Comment: delete ########## pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/AbortedTxnProcessor.java: ########## @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.transaction.buffer; + +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.client.api.transaction.TxnID; + + +public interface AbortedTxnProcessor { + + /** + * After the transaction buffer writes a transaction aborted marker to the topic, + * the transaction buffer will put the aborted txnID and the aborted marker position to AbortedTxnProcessor. + * @param txnID aborted transaction ID. + * @param position the position of the abort txnID + */ + void putAbortedTxnAndPosition(TxnID txnID, PositionImpl position); + + /** + * Clean up invalid aborted transactions. + */ + void trimExpiredAbortedTxns(); + + /** + * Check whether the transaction ID is an aborted transaction ID. + * @param txnID the transaction ID that needs to be checked. + * @param readPosition the read position of the transaction message, can be used to find the segment. + * @return a boolean, whether the transaction ID is an aborted transaction ID. + */ + boolean checkAbortedTransaction(TxnID txnID, Position readPosition); + + /** + * Recover transaction buffer by transaction buffer snapshot. + * @return a Position (startReadCursorPosition) determiner where to start to recover in the original topic. + */ + + CompletableFuture<PositionImpl> recoverFromSnapshot(); + + /** + * Delete the transaction buffer aborted transaction snapshot. + * @return a completableFuture. + */ + CompletableFuture<Void> deleteAbortedTxnSnapshot(); + + /** + * Take the frist snapshot if the topic has no snapshot before. + * @return a completableFuture. + */ + CompletableFuture<Void> takeAbortedTxnSnapshot(PositionImpl maxReadPosition); Review Comment: ```suggestion CompletableFuture<Void> takeAbortedTxnsSnapshot(PositionImpl maxReadPosition); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
