Closed Shawyeok closed 1 year ago
When use Key_Shared, if one consumer consume slowly and others are normal, MessageRedeliveryController#messagesToRedeliver will keep growing and could be very large.
Additional context:
The slow consumer will ends with zero availablePermits, and message positions will be recorded in MessageRedeliveryController#messagesToRedeliver
, when none of the whole batch of messages sent to consumer, isDispatcherStuckOnReplays
will be marked.
https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L310-L330
Next time call getMessagesToReplayNow
in readMoreEntries, it'll returns empty, then it'll read next batch of messages by cursor, therefor more message positions will be recorded.
https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L436-L446
https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L310-L330
@Shawyeok Can you attach the flame graph directly?
@Shawyeok Can you attach the flame graph directly?
@merlimat cpu_profile.html.zip attached
This is a very interesting issue to examine.
I think the time bound is likely O(n*log(n))
due to the TreeSet
generation (sorting) here:
https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java#L116
Edit:
We sort here to get the limit
of items (top few?) in the collection:
https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java#L113-L115
We then generate a TreeSet
, which is an LLRB Tree
, with a sorted list. This will rebalance the tree during each insert (expensive):
https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java#L116
I know there is typically an order guarantee within, but not across, partitions of topics in pub-sub systems. If there is no need for an ordered iteration through the set it might be possible to squeeze a little bit of performance out of using a HashSet
. You would need a custom comparator and equality implementation for the data type being inserted.
The issue had no activity for 30 days, mark with Stale label.
We also encounter such problem,topics in the same broker publish latency is very high. Any suggestions? @merlimat @codelipenghui
The issue should be fixed by https://github.com/apache/pulsar/pull/17804 @zhanghaou @Shawyeok
@codelipenghui Does the fix pr can be cherry pick to branch 2.8.X?
@zhanghaou We are only applying security patches for 2.8.x release for now.
@codelipenghui OK, thanks.
Describe the bug When use
Key_Shared
, if one consumer consume slowly and others are normal,MessageRedeliveryController#messagesToRedeliver
will keep growing and could be very large.MessageRedeliveryController#getMessagesToReplayNow
will be very expensive causemessagesToRedeliver.items()
is a O(n) operation in both time and space complexity. https://github.com/apache/pulsar/blob/2b2e0c50183c71e954f5c6e8bfcd7e36130279a5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/MessageRedeliveryController.java#L109-L122In this situation, one thread cpu usage could be 100%, this could effects the publish latency of other topics on the same broker instance.
To Reproduce Steps to reproduce the behavior:
Screenshots CPU flame graph
Additional context
2.8.1