YanshuoH opened a new issue, #22212: URL: https://github.com/apache/pulsar/issues/22212
### Search before asking - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar. ### Version Pulsar broker (and other components): v3.2.0 Pulsar client: go sdk v0.8.1 / v0.9.0 ### Minimal reproduce step As the incident happened in production environment, I've tried to simulate the conditions with the ideas of: 1. bookie slow response 2. broker pending requests high But unfortunately the increasing direct memory cannot be reproduced: 1. set broker config of `maxPendingPublishRequestsPerConnection` to 1 3. set client `OperationTimeout` and other timeout related parameters to a low value (like 100ms) 4. create a producer with high load and produce messages to `persistent://public/default/t1` 5. to simulate bookie slow response, either: 1. unload the topic 2. kill the bookie in charge of topic Then the two kinds of error occurred: 1. send operation: `2024/03/06 08:12:33 sendMessage failed: producer.Send: message send timeout: TimeoutError` 2. create new producer operation using the same connection: `ERRO[0012] Failed to create producer at send PRODUCER request error="request timed out" topic="persistent://public/default/t1-partition-0"` ### What did you expect to see? 1. DirectMemory should not have peak increasing 2. Producer should not encounter TimeoutError ### What did you see instead? 1. DirectMemory kept increasing 2. Producer send Timeout and new connection failed <img width="881" alt="37f52f7b-412b-4f1a-80c8-0c0e887f0094" src="https://github.com/apache/pulsar/assets/6973092/4d1b6d4b-58ff-4fb5-88a8-0c97bbb8afae"> ### Anything else? This issue is rather rare as we have used Pulsar for years and always kept the version fresh (from v2.2.x -> v3.2.0) and never have encountered such problem. I have digged into the source code, and for me, the possible DirectMemory reason may be that the `ledger.asyncAddEntry` did not complete (the async executor callback), so as code comment below, the buffer was not released. <img width="1011" alt="90419b98-ee35-4c80-a3ff-7308f6a105fb (1)" src="https://github.com/apache/pulsar/assets/6973092/8bebb053-4a88-4a1c-b3e7-b77019e9ee39"> Here are also some other findings that may help identify the root cause. 1. Throttled connection  The code that in charge of toggling `pulsar_broker_throttled_connections` is `org.apache.pulsar.broker.service.ServerCnx#startSendOperation` and `org.apache.pulsar.broker.service.ServerCnx#enableCnxAutoRead` That means during that period of time, `ServerCnx`'s pendingSendRequest should have reached the `maxPendingSendRequests` which is `1000` for our configuration. Also I can confirm that no publish rate limit set on any of the topics. 2. Metadata store executor queue  While the `disableCnxAutoRead` called, the command `Producer` on the connection will not be able to succeed so we can see many of below log: ``` 2024-03-04T09:29:56,302+0000 [pulsar-io-3-1] INFO org.apache.pulsar.broker.service.ServerCnx - [/10.120.159.82:59748] Closed producer before its creation was completed. producerId=61 ``` Then the producer will try to reconnect, which results to tons of metadata store operations in `org.apache.pulsar.broker.service.BrokerService#getOrCreateTopic`. 3. For what it's worth, the broker / bookie / zookeeper logs show no `Exception` or `error` and their CPU/Memory/JVM (except broker DirectMemory) seems fine. the peaks around 17:40 is the time that I've restarted the brokers on question. <img width="1716" alt="Screen Shot 2024-03-07 at 07 37 09" src="https://github.com/apache/pulsar/assets/6973092/317c88a2-c3cc-47d1-a7f2-3ac0971bc1d9"> ### Are you willing to submit a PR? - [ ] 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]
