-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[improve][broker] Reduce cpu usage of InMemoryDelayedDeliveryTracker. #24430
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
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 |
|---|---|---|
|
|
@@ -30,6 +30,7 @@ | |
| import java.util.NavigableSet; | ||
| import java.util.TreeSet; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import lombok.Getter; | ||
| import lombok.extern.slf4j.Slf4j; | ||
| import org.apache.bookkeeper.mledger.Position; | ||
|
|
@@ -64,6 +65,9 @@ public class InMemoryDelayedDeliveryTracker extends AbstractDelayedDeliveryTrack | |
| // The bit count to trim to reduce memory occupation. | ||
| private final int timestampPrecisionBitCnt; | ||
|
|
||
| // Count of delayed messages in the tracker. | ||
| private final AtomicLong delayedMessagesCount = new AtomicLong(0); | ||
|
|
||
| InMemoryDelayedDeliveryTracker(AbstractPersistentDispatcherMultipleConsumers dispatcher, Timer timer, | ||
| long tickTimeMillis, | ||
| boolean isDelayedDeliveryDeliverAtTimeStrict, | ||
|
|
@@ -125,6 +129,8 @@ public boolean addMessage(long ledgerId, long entryId, long deliverAt) { | |
| delayedMessageMap.computeIfAbsent(timestamp, k -> new Long2ObjectRBTreeMap<>()) | ||
| .computeIfAbsent(ledgerId, k -> new Roaring64Bitmap()) | ||
| .add(entryId); | ||
| delayedMessagesCount.incrementAndGet(); | ||
|
|
||
| updateTimer(); | ||
|
|
||
| checkAndUpdateHighest(deliverAt); | ||
|
|
@@ -183,13 +189,15 @@ public NavigableSet<Position> getScheduledMessages(int maxMessages) { | |
| positions.add(PositionFactory.create(ledgerId, entryId)); | ||
| }); | ||
| n -= cardinality; | ||
| delayedMessagesCount.addAndGet(-cardinality); | ||
| ledgerIdToDelete.add(ledgerId); | ||
| } else { | ||
| long[] entryIdsArray = entryIds.toArray(); | ||
| for (int i = 0; i < n; i++) { | ||
| positions.add(PositionFactory.create(ledgerId, entryIdsArray[i])); | ||
| entryIds.removeLong(entryIdsArray[i]); | ||
| } | ||
| delayedMessagesCount.addAndGet(-n); | ||
| n = 0; | ||
| } | ||
| if (n <= 0) { | ||
|
|
@@ -221,14 +229,13 @@ public NavigableSet<Position> getScheduledMessages(int maxMessages) { | |
| @Override | ||
| public CompletableFuture<Void> clear() { | ||
| this.delayedMessageMap.clear(); | ||
| this.delayedMessagesCount.set(0); | ||
|
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. On line 220 there's
Member
Author
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 is no need to do that, these logic is used for feature When the
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.
That's true. I think it would make sense to add an error log if delayedMessageCount isn't 0 at this point. That would help detect possible issues.
Member
Author
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. That is fine. Should i create a new PR to add the debug log?
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. @thetumbled I think it makes sense. It shouldn't be a debug log, but an error log if that ever happens. In addition, I think it should reset the count to 0, so that the state would be fixed.
Member
Author
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. Hi, Lari. I have created a pr to check this. But i think it should not be set to 0 because of the posibility of concurrent execution. |
||
| return CompletableFuture.completedFuture(null); | ||
| } | ||
|
|
||
| @Override | ||
| public long getNumberOfDelayedMessages() { | ||
| return delayedMessageMap.values().stream().mapToLong( | ||
| ledgerMap -> ledgerMap.values().stream().mapToLong( | ||
| Roaring64Bitmap::getLongCardinality).sum()).sum(); | ||
| return delayedMessagesCount.get(); | ||
thetumbled marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| /** | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.