-
Notifications
You must be signed in to change notification settings - Fork 3.6k
[improve][broker]Optimize InMemoryDelayedDeliveryTracker by maintaining state #23918
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 |
---|---|---|
|
@@ -218,9 +218,26 @@ public NavigableSet<Position> getScheduledMessages(int maxMessages) { | |
return positions; | ||
} | ||
|
||
public boolean shouldSkipMessage(long ledgerId, long entryId) { | ||
for (Long2ObjectMap<Roaring64Bitmap> ledgerMap : delayedMessageMap.values()) { | ||
Roaring64Bitmap entryIds = ledgerMap.get(ledgerId); | ||
if (entryIds != null && entryIds.contains(entryId)) { | ||
return true; | ||
} | ||
} | ||
return false; | ||
} | ||
|
||
@Override | ||
public CompletableFuture<Void> clear() { | ||
this.delayedMessageMap.clear(); | ||
long cutoffTime = getCutoffTime(); | ||
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. why only clear expired index? 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. @dao-jun The decision to clear only expired indices in the clear() method of the InMemoryDelayedDeliveryTracker is aimed at optimizing performance and maintaining the logical state of the message delivery system. By focusing on expired messages, we reduce the overhead associated with clearing and re-adding valid messages, which enhances performance, especially in scenarios with a high volume of delayed messages. This approach also allows us to retain the state of valid messages, enabling more efficient message delivery without needing to re-read them from storage. 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. I doubt that it will introduce memory leak problem. For example, we need to clear all delayed messages at method: IMO, the payoff of re-adding valid messages for once is acceptable, while the risk of such change is great. |
||
delayedMessageMap.headMap(cutoffTime).clear(); | ||
|
||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] Cleared expired delayed messages before {}, remaining messages: {}", | ||
dispatcher.getName(), cutoffTime, getNumberOfDelayedMessages()); | ||
} | ||
|
||
return CompletableFuture.completedFuture(null); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -193,12 +193,6 @@ public synchronized CompletableFuture<Void> addConsumer(Consumer consumer) { | |
shouldRewindBeforeReadingOrReplaying = false; | ||
} | ||
redeliveryMessages.clear(); | ||
delayedDeliveryTracker.ifPresent(tracker -> { | ||
// Don't clean up BucketDelayedDeliveryTracker, otherwise we will lose the bucket snapshot | ||
if (tracker instanceof InMemoryDelayedDeliveryTracker) { | ||
tracker.clear(); | ||
} | ||
}); | ||
} | ||
|
||
if (isConsumersExceededOnSubscription()) { | ||
|
@@ -448,7 +442,10 @@ protected Predicate<Position> createReadEntriesSkipConditionForNormalRead() { | |
// Filter out and skip read delayed messages exist in DelayedDeliveryTracker | ||
if (delayedDeliveryTracker.isPresent()) { | ||
final DelayedDeliveryTracker deliveryTracker = delayedDeliveryTracker.get(); | ||
if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { | ||
if (deliveryTracker instanceof InMemoryDelayedDeliveryTracker) { | ||
skipCondition = position -> ((InMemoryDelayedDeliveryTracker) deliveryTracker) | ||
.shouldSkipMessage(position.getLedgerId(), position.getEntryId()); | ||
} else if (deliveryTracker instanceof BucketDelayedDeliveryTracker) { | ||
skipCondition = position -> ((BucketDelayedDeliveryTracker) deliveryTracker) | ||
.containsMessage(position.getLedgerId(), position.getEntryId()); | ||
Comment on lines
-451
to
450
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. change |
||
} | ||
|
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.
it better to keep the method name with
BucketDelayedDeliveryTracker
, changeshouldSkipMessage
tocontains
.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.
@dao-jun ok