geniusjoe commented on issue #25021: URL: https://github.com/apache/pulsar/issues/25021#issuecomment-3605476419
> There a blog post by AutoMQ folks about PooledByteBufAllocator memory fragmentation: https://www.automq.com/blog/netty-based-streaming-systems-memory-fragmentation-and-oom-issues . I guess the blog post was written before [AdaptiveByteBufAllocator](https://netty.io/4.1/api/io/netty/buffer/AdaptiveByteBufAllocator.html) became usable. It would be interesting to also see how AdaptiveByteBufAllocator behaves in Pulsar. > > In Pulsar, the broker caching adds more of this fragmentation since it could hold on to much larger buffer when an entry is cached. In Netty, a slice of a buffer will hold the parent buffer in memory until all slices have been released. The mitigation in Pulsar is `managedLedgerCacheCopyEntries=true`, but that adds overhead since entries would get copied each time they are added to the cache. Lari thank you so much for your detailed reply. I also support adjusting the chunk size to 8MB (`-Dio.netty.allocator.maxOrder=10`) as a first step. At the very least, for scenarios involving large message payload (e.g., where each message is 4MB or larger), this change would prevent Netty from needing to call the `allocateHuge()` method to request direct memory from the OS for every single byteBuf allocation. Increasing the `maxOrder` should provide benefits for both message throughput and memory reuse. Your later point about adjusting the client-side JVM startup parameters to improve throughput efficiency is also very meaningful. I will try enabling this parameter in some of our high-traffic client clusters to verify if performance improves. In our scenario, we have also observed the backpressure mentioned in your issue #24926. When a topic has many subscriptions, and each subscription has only one active consumer with a large `receiver_queue_size`(default: 1000 entries), slow network speeds can easily trigger the channel’s high watermark(`c.isWritable() = false`). Since no other consumer channels are available for writing within the current subscription, the consumer’s dispatcher will eventually write all entries from the read batch, one by one, into this single channel. This leads to increased direct memory usage. To mitigate this, we have reduced the client-side queue size(to 20) to lower direct memory consumption. However, a long-term solution may require optimizing the `trySendMessagesToConsumers` logic for high-watermark scenarios. Due to varying write speeds across different channels, the timing of `byteBuf` releases also differs. This situation is highly likely to cause memory fragmentation within chunks. In one of my clusters, each broker runs on a 16C32G spec, configured with a 12GB heap and 16GB of direct memory (`-Xms12g -Xmx12g -XX:MaxDirectMemorySize=16g`). The current Netty configuration uses `-Dio.netty.allocator.maxOrder=13, -Dio.netty.allocator.numDirectArenas=8, and -Dio.netty.allocator.maxCachedBufferCapacity=8388608`. During production operation, some brokers maintain consistently high direct memory usage, typically ranging from 8–11GB. However, despite this high usage, after observing for a week, I have not detected a clear upward trend in direct memory consumption, nor have I observed the memory leak issue described in the AutoMQ documentation. And in the long term, it might be valuable enough to conduct a performance comparison or benchmark between the `AdaptiveByteBufAllocator` and the current `PooledByteBufAllocator`. <img width="3682" height="1468" alt="Image" src="https://github.com/user-attachments/assets/2e86b035-21b8-47ab-ad3d-ff5c67741b8e" /> This Pulsar cluster runs only one Pulsar broker per pod. The configured 16GB of direct memory is dedicated exclusively to this single broker. Therefore, my primary concern isn't the reclamation speed of the chunks holding the byteBufs. Instead, I aim to maximize the chunk reuse rate to avoid frequent allocation and deallocation of native memory by Netty from the OS. I probably won't enable the `managedLedgerCacheCopyEntries=true` configuration currently. -- 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]
