-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[fix][broker] Move pending acks cleanup to selected mark-delete callbacks #25592
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -145,7 +145,6 @@ public class PersistentDispatcherMultipleConsumers extends AbstractPersistentDis | |
| protected enum ReadType { | ||
| Normal, Replay | ||
| } | ||
| private Position lastMarkDeletePositionBeforeReadMoreEntries; | ||
| private volatile long readMoreEntriesCallCount; | ||
|
|
||
| public PersistentDispatcherMultipleConsumers(PersistentTopic topic, ManagedCursor cursor, | ||
|
|
@@ -362,17 +361,6 @@ public synchronized void readMoreEntries() { | |
| // increment the counter for readMoreEntries calls, to track the number of times readMoreEntries is called | ||
| readMoreEntriesCallCount++; | ||
|
|
||
| // remove possible expired messages from redelivery tracker and pending acks | ||
| Position markDeletePosition = cursor.getMarkDeletedPosition(); | ||
| if (lastMarkDeletePositionBeforeReadMoreEntries != markDeletePosition) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Another detail is that trimming (based on retention) doesn't currently trigger the callbacks at all. Trimming is handled by the org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl#internalTrimLedgers method which calls advanceCursorsIfNecessary (btw. The javadoc of the advanceCursorsIfNecessary method is misleading. For some reason, the javadoc added in https://github.com/apache/pulsar/pull/10667/changes hasn't made it to the code). Trimming based on retention should also be handled before we can remove this code.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Even the current code doesn't currently handle trimming correctly since trimming should trigger state cleanup proactively. However, it would cleanup state correctly. (In some cases that's sufficient, in some cases it's not)
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There should be a callback for ManagedCursor for deletions that happen by retention trimming. |
||
| redeliveryMessages.removeAllUpTo(markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); | ||
| for (Consumer consumer : consumerList) { | ||
| consumer.removePendingAcksUpToPositionAndDecrementUnacked( | ||
| markDeletePosition.getLedgerId(), markDeletePosition.getEntryId()); | ||
| } | ||
| lastMarkDeletePositionBeforeReadMoreEntries = markDeletePosition; | ||
| } | ||
|
|
||
| // totalAvailablePermits may be updated by other threads | ||
| int firstAvailableConsumerPermits = getFirstAvailableConsumerPermits(); | ||
| int currentTotalAvailablePermits = Math.max(totalAvailablePermits, firstAvailableConsumerPermits); | ||
|
|
@@ -600,6 +588,14 @@ public CopyOnWriteArrayList<Consumer> getConsumers() { | |
| return consumerList; | ||
| } | ||
|
|
||
| @Override | ||
| public void prunePendingAcksUpToPosition(long ledgerId, long entryId) { | ||
| redeliveryMessages.removeAllUpTo(ledgerId, entryId); | ||
| for (Consumer consumer : consumerList) { | ||
| consumer.removePendingAcksUpToPositionAndDecrementUnacked(ledgerId, entryId); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized boolean canUnsubscribe(Consumer consumer) { | ||
| return consumerList.size() == 1 && consumerSet.contains(consumer); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,6 +38,7 @@ | |
| import org.apache.bookkeeper.mledger.PositionFactory; | ||
| import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; | ||
| import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; | ||
| import org.apache.pulsar.broker.service.Dispatcher; | ||
| import org.apache.pulsar.broker.service.MessageExpirer; | ||
| import org.apache.pulsar.client.impl.MessageImpl; | ||
| import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; | ||
|
|
@@ -211,9 +212,18 @@ public void markDeleteComplete(Object ctx) { | |
| long numMessagesExpired = (long) ctx - cursor.getNumberOfEntriesInBacklog(false); | ||
| msgExpired.recordMultipleEvents(numMessagesExpired, 0 /* no value stats */); | ||
| totalMsgExpired.add(numMessagesExpired); | ||
| // If the subscription is a Key_Shared subscription, we should to trigger message dispatch. | ||
| if (subscription != null && subscription.getType() == SubType.Key_Shared) { | ||
| subscription.getDispatcher().markDeletePositionMoveForward(); | ||
| if (subscription != null) { | ||
| Dispatcher dispatcher = subscription.getDispatcher(); | ||
| if (dispatcher != null) { | ||
| Position mdPos = cursor.getMarkDeletedPosition(); | ||
| if (mdPos != null) { | ||
| dispatcher.prunePendingAcksUpToPosition(mdPos.getLedgerId(), mdPos.getEntryId()); | ||
| } | ||
| // If the subscription is a Key_Shared subscription, we should to trigger message dispatch. | ||
| if (subscription.getType() == SubType.Key_Shared) { | ||
| dispatcher.markDeletePositionMoveForward(); | ||
| } | ||
|
Comment on lines
+218
to
+225
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. instead of adding a separate |
||
| } | ||
| } | ||
| expirationCheckInProgress = FALSE; | ||
| log.debug() | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
instead of adding a separate
prunePendingAcksUpToPosition, leave the pruning as an implementation of the dispatcher. Modify the existingmarkDeletePositionMoveForwardso that it contains the markDeletedPosition argument. In the implementation, the pruning can be handled for Key_Shared.