pulsar icon indicating copy to clipboard operation
pulsar copied to clipboard

[Enhancement] Improve Pulsar Broker cache defaults to get better out-of-the-box performance

Open lhotari opened this issue 1 year ago • 8 comments

Search before asking

  • [X] I searched in the issues and found nothing similar.

Mailing list discussion thread: https://lists.apache.org/thread/5od69114jfrzo9dkbllxycq8o7ns341y

Motivation

It's crucial to tune the Pulsar broker cache since the defaults in Pulsar are not optimal. Besides poor performance for Pulsar use cases, this leads to wasted CPU and unnecessary network transfer costs in cloud environments. Tuning the Pulsar broker cache improves performance and reduces costs, especially with high fan-out use cases, Key_Shared subscriptions, and tiered storage.

Solution

Here are some settings which would be better defaults.

  • maxMessagePublishBufferSizeInMB - not broker cache related, but it's necessary to set it to an explicit value when fine-tuning broker cache settings so that direct memory OOM can be avoided. Default is 50% of direct memory. Set to 500
  • managedLedgerCacheSizeMB - the default is 20% of direct memory. It's better to set it to an explicit value to avoid direct memory OOM. Set to 512
  • managedLedgerMaxReadsInFlightSizeInMB - this feature is disabled by default. It's useful for avoiding direct memory OOM, which is a known issue with the default dispatcherDispatchMessagesInSubscriptionThread=true setting unless managedLedgerMaxReadsInFlightSizeInMB is set. Set to 500. The value should be higher than dispatcherMaxReadBatchSize * maxMessageSize.
  • managedLedgerCacheEvictionTimeThresholdMillis - the default 1000 is too low. Set to 10000
  • managedLedgerCacheEvictionIntervalMs - the default 10 is too low. Set to 5000 to avoid spending a lot of CPU with cache eviction.
  • managedLedgerMinimumBacklogCursorsForCaching - the default 0 disables backlog cursors (catch-up read) caching. Set to 3
  • managedLedgerMinimumBacklogEntriesForCaching - the default 1000 is way too high. Set to 1
  • managedLedgerMaxBacklogBetweenCursorsForCaching - the default 10000 is way too low. Set to 2147483647 to disable the limit completely.

Sample settings for broker cache tuning:

yaml format:

  maxMessagePublishBufferSizeInMB: 500
  managedLedgerCacheSizeMB: 512
  managedLedgerMaxReadsInFlightSizeInMB: 500
  managedLedgerCacheEvictionTimeThresholdMillis: 10000
  managedLedgerCacheEvictionIntervalMs: 5000
  managedLedgerMinimumBacklogCursorsForCaching: 3
  managedLedgerMinimumBacklogEntriesForCaching: 1
  managedLedgerMaxBacklogBetweenCursorsForCaching: 2147483647

broker.conf format:

maxMessagePublishBufferSizeInMB=500
managedLedgerCacheSizeMB=512
managedLedgerMaxReadsInFlightSizeInMB=500
managedLedgerCacheEvictionTimeThresholdMillis=10000
managedLedgerCacheEvictionIntervalMs=5000
managedLedgerMinimumBacklogCursorsForCaching=3
managedLedgerMinimumBacklogEntriesForCaching=1
managedLedgerMaxBacklogBetweenCursorsForCaching=2147483647

managedLedgerMaxReadsInFlightSizeInMB will have to be set to value that is higher than dispatcherMaxReadBatchSize * maxMessageSize. Otherwise it could result in error Time-out elapsed while acquiring enough permits on the memory limiter to read from ledger [ledgerid], [topic], estimated read size [read size] bytes for [dispatcherMaxReadBatchSize] entries (check managedLedgerMaxReadsInFlightSizeInMB). dispatcherMaxReadBatchSize defaults to 100 and maxMessageSize defaults to 5MB in bytes. There's a separate issue to address the problem when managedLedgerMaxReadsInFlightSizeInMB < ``dispatcherMaxReadBatchSize*maxMessageSize`, https://github.com/apache/pulsar/issues/23482.

Alternatives

No response

Anything else?

The broker cache hit rate can be monitored with the Grafana dashboard found at https://github.com/datastax/pulsar-helm-chart/blob/master/helm-chart-sources/pulsar/grafana-dashboards/broker-cache-by-broker.json (Apache 2.0 license). The broker cache also impacts offloading. Offloading can be monitored with the dashboard available at https://github.com/apache/pulsar/blob/master/grafana/dashboards/offloader.json .

Are you willing to submit a PR?

  • [ ] I'm willing to submit a PR!

lhotari avatar Oct 16 '24 07:10 lhotari

the changes to maxMessagePublishBufferSizeInMB and managedLedgerCacheSizeMB should be highlighted strongly in the release notes. They can even be considered as breaking changes. Most of the production systems might be relying on the default values. In broker pods with available memory of 4GB or above, this would lead to a change of max publish buffer of 2+GB to 500MB - this is a big change and might actually slip in silently, leading to degradation of the system.

the cache eviction config changes look good.

grssam avatar Oct 16 '24 08:10 grssam

the changes to maxMessagePublishBufferSizeInMB and managedLedgerCacheSizeMB should be highlighted strongly in the release notes. They can even be considered as breaking changes. Most of the production systems might be relying on the default values. In broker pods with available memory of 4GB or above, this would lead to a change of max publish buffer of 2+GB to 500MB - this is a big change and might actually slip in silently, leading to degradation of the system.

@grssam I agree. It's just that the current defaults are not optimal at all for most users. Users that fine tune configurations, should set the values explicitly. It could be helpful to add a way to have dynamic setting with lower and upper bounds. Perhaps supporting an expression language like SpEL or MVEL for evaluating the dynamic expression could be useful.

the cache eviction config changes look good.

I also added more context for managedLedgerMaxReadsInFlightSizeInMB. This feature is disabled by default. It's useful for avoiding direct memory OOM, which is a known issue with the default dispatcherDispatchMessagesInSubscriptionThread=true setting unless managedLedgerMaxReadsInFlightSizeInMB is set.

lhotari avatar Oct 16 '24 08:10 lhotari

The broker cache also impacts offloading. Offloading can be monitored with the dashboard available at https://github.com/streamnative/apache-pulsar-grafana-dashboard/pull/74 .

@dao-jun is planning to contribute the offloading dashboard to Apache Pulsar project so that it could be included in https://github.com/apache/pulsar/tree/master/grafana/dashboards.

lhotari avatar Oct 16 '24 17:10 lhotari

The broker cache also impacts offloading. Offloading can be monitored with the dashboard available at https://github.com/streamnative/apache-pulsar-grafana-dashboard/pull/74 .

@dao-jun is planning to contribute the offloading dashboard to Apache Pulsar project so that it could be included in https://github.com/apache/pulsar/tree/master/grafana/dashboards.

This is completed by #23479 , thank you @dao-jun. dashboard is now available at https://github.com/apache/pulsar/blob/master/grafana/dashboards/offloader.json .

lhotari avatar Oct 18 '24 06:10 lhotari

An additional consideration is to set cacheEvictionByMarkDeletedPosition=true to prevent issue described in #16421

lhotari avatar Nov 05 '24 23:11 lhotari

Most recent status update which touches upon managedLedgerMaxReadsInFlightSizeInMB bugs: https://github.com/apache/pulsar/pull/23869#issuecomment-2609121174

lhotari avatar Jan 23 '25 08:01 lhotari

An additional consideration is to set cacheEvictionByMarkDeletedPosition=true to prevent issue described in #16421

There's currently an open issue #23900 where the broker cache is always by-passed for replay queue reads, also when cacheEvictionByMarkDeletedPosition=true is used. That's addressed by #23903 . The limitation and issue described in #16421 remains. I have work in progress to address that issue.

lhotari avatar Jan 28 '25 16:01 lhotari

Broker entry cache eviction (and retention) improvements: https://github.com/apache/pulsar/pull/24363, foundation for further improvements

lhotari avatar May 30 '25 06:05 lhotari

"PIP-430: Pulsar Broker cache improvements: refactoring eviction and adding a new cache strategy based on expected read count", in #24444

lhotari avatar Jun 23 '25 18:06 lhotari

PIP-430 coming in Pulsar 4.1.0 will be solving most of the cache issues.

A tuning advice in one of the comments which could also help in existing Pulsar versions: https://github.com/apache/pulsar/pull/24623#issuecomment-3249499948

lhotari avatar Sep 03 '25 14:09 lhotari

Addressed by PIP-430 in Pulsar 4.1.x.

lhotari avatar Dec 09 '25 12:12 lhotari