codelipenghui opened a new pull request, #17804: URL: https://github.com/apache/pulsar/pull/17804
### Motivation The implementation of MessageRedeliveryController is inefficient - Duplicated Ledger IDs https://github.com/apache/pulsar/blob/master/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentSortedLongPairSet.java#L50 - high sorting overhead even if you just want to get a few redeliveries https://github.com/apache/pulsar/compare/master...codelipenghui:incubator-pulsar:penghui/improve-relay-queue#diff-4c2d35368b21e5c33d65886196e8673c04a69f5c5cb888cdd04c29f8823502fdL112-L116 <img width="1842" alt="image" src="https://user-images.githubusercontent.com/12592133/191750398-61caba7e-6056-48f5-9932-3ed59ecf9ee9.png"> ``` "BookKeeperClientWorker-OrderedExecutor-1-0" #40 prio=5 os_prio=0 cpu=27401742.31ms elapsed=788038.50s tid=0x00007f4cf95bc800 nid=0x56 runnable [0x00007f4cd49a0000] java.lang.Thread.State: RUNNABLE at java.util.TreeMap.put([email protected]/TreeMap.java:566) at java.util.TreeSet.add([email protected]/TreeSet.java:255) at org.apache.pulsar.common.util.collections.ConcurrentSortedLongPairSet.lambda$items$4(ConcurrentSortedLongPairSet.java:152) at org.apache.pulsar.common.util.collections.ConcurrentSortedLongPairSet$$Lambda$1150/0x00000008408d9040.accept(Unknown Source) at org.apache.pulsar.common.util.collections.ConcurrentLongPairSet$Section.forEach(ConcurrentLongPairSet.java:563) at org.apache.pulsar.common.util.collections.ConcurrentLongPairSet.forEach(ConcurrentLongPairSet.java:242) at org.apache.pulsar.common.util.collections.ConcurrentSortedLongPairSet.items(ConcurrentSortedLongPairSet.java:151) at org.apache.pulsar.common.util.collections.ConcurrentSortedLongPairSet.items(ConcurrentSortedLongPairSet.java:143) at org.apache.pulsar.common.util.collections.ConcurrentSortedLongPairSet.items(ConcurrentSortedLongPairSet.java:128) at org.apache.pulsar.broker.service.persistent.MessageRedeliveryController.getMessagesToReplayNow(MessageRedeliveryController.java:112) at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.getMessagesToReplayNow(PersistentDispatcherMultipleConsumers.java:850) - eliminated <0x0000000757da7548> (a org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers) at org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.getMessagesToReplayNow(PersistentStickyKeyDispatcherMultipleConsumers.java:432) - locked <0x0000000757da7548> (a org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers) at org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.java:175) at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readEntriesComplete(PersistentDispatcherMultipleConsumers.java:483) - locked <0x0000000757da7548> (a org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers) at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$11.readEntryComplete(ManagedCursorImpl.java:1326) - locked <0x00000007129d8210> (a org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$11) at org.apache.bookkeeper.mledger.impl.EntryCacheImpl.lambda$asyncReadEntry0$0(EntryCacheImpl.java:222) at org.apache.bookkeeper.mledger.impl.EntryCacheImpl$$Lambda$1158/0x00000008408db440.accept(Unknown Source) at java.util.concurrent.CompletableFuture$UniAccept.tryFire([email protected]/CompletableFuture.java:714) at java.util.concurrent.CompletableFuture$Completion.run([email protected]/CompletableFuture.java:478) at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1128) at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:628) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run([email protected]/Thread.java:829) ``` A related fix #15354, but the fix can't fix the case that the `MessageRedeliveryController` have many redelivery messages This PR can make the `MessageRedeliveryController` work more efficiently by introducing a new Ordered Map and Bitmap based LongPair Ordered Set ### Modifications - Added [RoaringBitmap](https://roaringbitmap.org/) dependency which is a widely used compressed bitset. The dependency is introduced to broker, not the common module because the common module will also expose the Java Client - Added BitmapSortedLongPairSet for MessageRedeliveryController - Update the MessageRedeliveryController to use BitmapSortedLongPairSet for messagesToRedeliver ### Verifying this change New test added BitmapSortedLongPairSetTest ### Does this pull request potentially affect one of the following parts: *If the box was checked, please highlight the changes* - [ ] Dependencies (add or upgrade a dependency) - [ ] The public API - [ ] The schema - [ ] The default values of configurations - [ ] The binary protocol - [ ] The REST endpoints - [ ] The admin CLI options - [ ] Anything that affects deployment ### Documentation <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. --> - [ ] `doc-required` (Your PR needs to update docs, and you will update later) - [x] `doc-not-needed` (Please explain why) - [ ] `doc` (Your PR contains doc changes) - [ ] `doc-complete` (Docs have been already added) ### Matching PR in forked repository PR in forked repository: https://github.com/codelipenghui/incubator-pulsar/pull/14 -- 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]
