Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2061,23 +2061,28 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId)
long requestId = getLastMessageId.getRequestId();

Topic topic = consumer.getSubscription().getTopic();
Position lastPosition = topic.getLastPosition();
int partitionIndex = TopicName.getPartitionIndex(topic.getName());

Position markDeletePosition = null;
if (consumer.getSubscription() instanceof PersistentSubscription) {
markDeletePosition = ((PersistentSubscription) consumer.getSubscription()).getCursor()
.getMarkDeletedPosition();
}

getLargestBatchIndexWhenPossible(
topic,
(PositionImpl) lastPosition,
(PositionImpl) markDeletePosition,
partitionIndex,
requestId,
consumer.getSubscription().getName());
topic.checkIfTransactionBufferRecoverCompletely(true).thenRun(() -> {
Position lastPosition = ((PersistentTopic) topic).getMaxReadPosition();
int partitionIndex = TopicName.getPartitionIndex(topic.getName());

Position markDeletePosition = null;
if (consumer.getSubscription() instanceof PersistentSubscription) {
markDeletePosition = ((PersistentSubscription) consumer.getSubscription()).getCursor()
.getMarkDeletedPosition();
}

getLargestBatchIndexWhenPossible(
topic,
(PositionImpl) lastPosition,
(PositionImpl) markDeletePosition,
partitionIndex,
requestId,
consumer.getSubscription().getName());
}).exceptionally(e -> {
writeAndFlush(Commands.newError(getLastMessageId.getRequestId(),
ServerError.UnknownError, "Failed to recover Transaction Buffer."));
return null;
});
} else {
writeAndFlush(Commands.newError(getLastMessageId.getRequestId(),
ServerError.MetadataError, "Consumer not found"));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -332,7 +332,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS
this.transactionBuffer = brokerService.getPulsar()
.getTransactionBufferProvider().newTransactionBuffer(this);
} else {
this.transactionBuffer = new TransactionBufferDisable();
this.transactionBuffer = new TransactionBufferDisable(this);
}
transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry());
if (ledger instanceof ShadowManagedLedgerImpl) {
Expand Down Expand Up @@ -421,7 +421,7 @@ public CompletableFuture<Void> initialize() {
this.transactionBuffer = brokerService.getPulsar()
.getTransactionBufferProvider().newTransactionBuffer(this);
} else {
this.transactionBuffer = new TransactionBufferDisable();
this.transactionBuffer = new TransactionBufferDisable(this);
}
shadowSourceTopic = null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -211,9 +211,12 @@ public TransactionBufferReader newReader(long sequenceId) throws

final ConcurrentMap<TxnID, TxnBuffer> buffers;
final Map<Long, Set<TxnID>> txnIndex;
private final Topic topic;

public InMemTransactionBuffer(Topic topic) {
this.buffers = new ConcurrentHashMap<>();
this.txnIndex = new HashMap<>();
this.topic = topic;
}

@Override
Expand Down Expand Up @@ -371,7 +374,7 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position) {

@Override
public PositionImpl getMaxReadPosition() {
return PositionImpl.LATEST;
return (PositionImpl) topic.getLastPosition();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -443,8 +443,7 @@ void updateMaxReadPosition(TxnID txnID) {
ongoingTxns.remove(txnID);
if (!ongoingTxns.isEmpty()) {
PositionImpl position = ongoingTxns.get(ongoingTxns.firstKey());
//max read position is less than first ongoing transaction message position, so entryId -1
maxReadPosition = PositionImpl.get(position.getLedgerId(), position.getEntryId() - 1);
maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position);
} else {
maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.Topic;
import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer;
import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader;
import org.apache.pulsar.broker.transaction.buffer.TransactionMeta;
Expand All @@ -38,6 +39,10 @@
*/
@Slf4j
public class TransactionBufferDisable implements TransactionBuffer {
private final Topic topic;
public TransactionBufferDisable(Topic topic) {
this.topic = topic;
}

@Override
public CompletableFuture<TransactionMeta> getTransactionMeta(TxnID txnID) {
Expand Down Expand Up @@ -90,7 +95,7 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position) {

@Override
public PositionImpl getMaxReadPosition() {
return PositionImpl.LATEST;
return (PositionImpl) topic.getLastPosition();
}

@Override
Expand Down