apache / pulsar

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org/
Apache License 2.0
14.19k stars 3.58k forks source link

CPU usage 100% when use Key_Shared #15445

Closed Shawyeok closed 1 year ago

Shawyeok commented 2 years ago

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 cause messagesToRedeliver.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-L122

In 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:

  1. start a standalone cluster
  2. start a producer
    $ bin/pulsar-perf produce -mk random persistent://public/default/test
  3. start a slow consumer and a normal consumer
    $ bin/pulsar-perf consume -st Key_Shared persistent://public/default/test
    $ bin/pulsar-perf consume -st Key_Shared -r 0.1 persistent://public/default/test

Screenshots CPU flame graph image

Additional context

Shawyeok commented 2 years 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

merlimat commented 2 years ago

@Shawyeok Can you attach the flame graph directly?

Shawyeok commented 2 years ago

@Shawyeok Can you attach the flame graph directly?

@merlimat cpu_profile.html.zip attached

surahman commented 2 years ago

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.

github-actions[bot] commented 2 years ago

The issue had no activity for 30 days, mark with Stale label.

zhanghaou commented 1 year ago

We also encounter such problem,topics in the same broker publish latency is very high. Any suggestions? @merlimat @codelipenghui

codelipenghui commented 1 year ago

The issue should be fixed by https://github.com/apache/pulsar/pull/17804 @zhanghaou @Shawyeok

zhanghaou commented 1 year ago

@codelipenghui Does the fix pr can be cherry pick to branch 2.8.X?

codelipenghui commented 1 year ago

@zhanghaou We are only applying security patches for 2.8.x release for now.

zhanghaou commented 1 year ago

@codelipenghui OK, thanks.