geniusjoe opened a new issue, #25021: URL: https://github.com/apache/pulsar/issues/25021
### Search before reporting - [x] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar. ### Read release policy - [x] I understand that [unsupported versions](https://pulsar.apache.org/contribute/release-policy/#supported-versions) don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker. ### User environment Broker version: 4.0 LTS Broker Operating system and hardware type: 5.4.241-1-tlinux4 (cloud vendor host Linux distributions, acted like CentOS) Broker Java version: jdk 21 Client library type: Java Client library version: apache-pulsar-3.0.4 Client Operating system and hardware type: 5.4.241-1-tlinux4 Client Java version: openjdk version "17.0.12" 2024-07-23 LTS ### Issue Description One of our maintained Pulsar instances's brokers (with a 16C32G spec.) frequently experience physical memory OOM, leading to restarts. However, the memory usage shown in the `jcmd <pid> native_memory summary` does not match the `top` command process RES usage. Typically, just before an OOM occurs, top shows memory usage of around 31GB, while the total memory usage in the `jcmd <pid> native_memory summary` is only about 20GB. Pulsar broker restarts due to OOM is abnormal and need to find out the reason. <img width="2612" height="1208" alt="Image" src="https://github.com/user-attachments/assets/4eeb1209-ebcb-4b62-977f-6e97db3df51d" /> ### Error messages ```text Process was killed by K8S OOMKiller and print error log in `kubectl describe pod` command Last State section: `CGroup OOM encountered, victim process: java, pid: <broker-pid>` ``` ### Reproducing the issue I believe OOM scenario is related to the broker frequently reading large-sized messages from bookies and then quickly release them. This scenario is more likely to occurred in broadcast mode (with high filter ratio and large amount of subscriptions). Below is one of reproduce ways: 1. Use 1 broker with 8C16G 5Gbps NIC hardware and 3 bookie, configure `E:Qw:Qa = 2:2:1` for improving bookie's message throughput 2. Create approximately 1000 subscriptions, each subscription only creates one consumer, and add unique properties to each subscription 3. Send a 1MB message every 1 second, and each message's properties matching only one random subscription among the 1000 subscriptions 4. Configure broker EntryFilters to ensure that each message will only be sent to one subscription that matches it, while all other subscriptions cannot receive the message 5. Make sure broker ledger cache is always missed ### Additional information I believe this issue is mainly related to the memory allocation mechanism of Netty's ByteBuf allocator: 1. When message size is large with high filtering ratio, due to Pulsar's default configuration of `dispatcherMaxReadSizeBytes=5MB` and `dispatcherMaxBatchSize=100`, a single `readMoreEntries()` batch can easily reach the maximum read size limit of 5MB. With the current `E:Qw:Qa = 2:2:1` mode, a single read operation only requests from two channel eventLoop threads. Since the default netty chunk size is 4MB(`DEFAULT_PAGE_SIZE << DEFAULT_MAX_ORDER`), it's easy for the existing chunks in the netty `poolArena` to have insufficient space, requiring allocation of new chunks from native memory. 2. The newly allocated chunks might be fully occupied by current `readMoreEntries()` operation. Due to the high message filtering ratio, it's very likely that messages read in this batch are all filtered out and not delivered to consumers, causing the entry byteBufs to be quickly released in the broker almost at the same time. 3. Netty default threadLocal cache size `maxCachedBufferCapacity` is 32KB. When a single byteBuf exceeds the cache size, it cannot be cached in threadLocal but directly released and returned to the chunk. When all entries read in this batch are larger than 32KB and released at the same time, it will triggering Netty's chunk recycle strategy that free chunk and give back memory to OS. And I noticed netty allocate and release memory very often in jfr: <img width="3840" height="2076" alt="Image" src="https://github.com/user-attachments/assets/bf14fd59-383e-438a-ba84-902f8d049c33" /> 4. This issue might be related to the implementation of JDK runtime, Linux distribution, or glibc different `malloc()` realization. However, I believe it should be a common problem. In my test environment, when the JDK allocates direct memory smaller than 64MB (i.e., each chunk allocation is 4MB), using command `pmap -x <broker-pid> | sort -n -k3` reveals that the OS always requests a memory segment of 61444 KB. When chunks are too frequently allocated and released (maybe at a rate 20/s), I find that the OS memory cleanup speed cannot keep up with the allocation speed. Which results in the native memory size seen via the `top` command being significantly larger than in the `jcmd <broker-pid> VM.native_memory summary`.<img width="2797" height="997" alt="Image" src="https://github.com/user-attachments/assets/51bf07b9-b505-44e1-8ece-212463332687" /> To solve this issue, maybe we can update below netty related options: ``` -Dio.netty.allocator.maxOrder=13 -Dio.netty.allocator.maxCachedBufferCapacity=8388608``` 1. The `maxOrder` parameter adjusts the maximum size of a single chunk. Changing the default from 4M to 64M increases the number of byteBufs that can be stored in a single chunk. Furthermore, after increasing the single chunk size to 64MB, it can accommodate a default maximum 5MB single message. Netty changed the default chunk size from 16M to 4M in version 4.1.76. The primary reason was that the Netty community believed that in most scenarios, each PoolArena does not require very large chunk sizes, as it could easily lead to wasted memory resources in this pr https://github.com/netty/netty/pull/12108: "_If we don't need a lot of memory, and certainly not compared to the number of cores on a system, then this will take up more memory than necessary, since each chunk is 16 MiB._", which I think is not very suitable for pulsar use scene. In practice, the Logstash community also already reverted it back to the original default value of 16MB. https://github.com/elastic/logstash/issues/15765 2. Adjusting the `maxCachedBufferCapacity` parameter increases the maximum size of a single thread-local cache from the default 32KB to 8MB. As a result, the byteBuf corresponding to the default maximum 5MB message can now be cached in the thread-local cache, thereby extending the lifecycle of chunks. Additionally, increasing the cache size reduces the frequency of memory allocation requests by pulsar eventGroup thread-local caches from the PoolArena, which in turn decreases lock waiting time during concurrent memory allocation in the PoolArena. After adjusting the Netty options mentioned above, I observed a significant reduction in Netty native memory allocation events shown in JFR (Java Flight Recorder), with the frequency dropping to about one allocation and deallocation every 1.5 minutes. Furthermore, the memory usage reported in the JVM's `jcmd <broker-pid> native_memory summary` now almost is the same as which displayed by the `top` command. I think under current configuration, the operating system can have enough time to recycle native memory. <img width="3792" height="1890" alt="Image" src="https://github.com/user-attachments/assets/8b2982b5-f10c-4137-a03b-772e89143dd7" /> ### Are you willing to submit a PR? - [x] I'm willing to submit a PR! -- 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]
