chamons opened a new issue, #25617: URL: https://github.com/apache/pulsar/issues/25617
### 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 ``` $ bin/pulsar version Current version of pulsar is: 4.1.2 Git Revision 5f587c1204d13ec6b657c7eef472f8427368a9f6 Git Branch branch-4.1 Built by Lari Hotari <[email protected]> on Laris-MBP-2.lan at 2025-11-11T23:36:57+0200 ``` Broker OS: ``` $ uname -a Linux pulsar-broker-0 6.6.113+ #1 SMP Wed Jan 28 03:26:59 UTC 2026 aarch64 GNU/Linux ``` Broker Java: ``` $ java -version openjdk version "21.0.9" 2025-10-21 LTS OpenJDK Runtime Environment Corretto-21.0.9.11.1 (build 21.0.9+11-LTS) OpenJDK 64-Bit Server VM Corretto-21.0.9.11.1 (build 21.0.9+11-LTS, mixed mode) ``` Client Library: https://github.com/streamnative/pulsar-rs ``` pulsar = { version = "6.7.2", features = [ "compression", "tokio-runtime", ], default-features = false } ``` Client OS Version - Same as Broker OS ### Issue Description We have a production broker and bookie cluster of 6 nodes each. While scaling up a new feature of our service that wrote high volume messages to a topic with a deliver_at_time mostly set for the upcoming minute, we saw this exposed pulsar metric increase over time: `sum by (topic)(pulsar_subscription_back_log_no_delayed{namespace=~"pulsar-$environment", subscription="delivery-subscription"})` <img width="1413" height="313" alt="Image" src="https://github.com/user-attachments/assets/5b32e909-55ed-4edc-907c-07522a949474" /> This was despite the fact that our consumers were consuming from that topic actively and we had plenty of headroom to consume more messages from our consumers. After an investigation, we restarted our consumer pool to no effect, and then restarted the brokers (a rolling restart). We then saw a _massive_ spike from 42.7 thousand to to 2.64 _million_ entries on that topic, which then started to be processed by our consumer group. <img width="1411" height="304" alt="Image" src="https://github.com/user-attachments/assets/750a2ecc-bd61-40f2-be95-12c24e4583a5" /> Investigating the messages now being sent, we found messages from more than a week ago, including some that we have evidence were previously processed and acked successfully. Thankfully we had added a stale message detection to our consumer, so this was only a data loss event and not something worse (sending messages from a week ago). Investigating the logs, we found 165 instances of java.util.NoSuchElementException coming from Long2ObjectAVLTreeMap/InMemoryDelayedDeliveryTracker (see error message attached), with some of those correlating pretty closely to increases in the "stuck" backlog or high volume writes to pulsar. There are three bugs here from my perspective: - Why did the broker cluster stop dispatching messages that had passed their deliver_at_time for this topic until the broker was restarted? - Why did restarting the broker cause multiple order of magnitude more messages to be (re)? send to our consumers? - Why did pulsar have uncaught exceptions that appear to be the source of this bug only reported by an _info_ level log? Why did the broker not fail a health check or otherwise notify us via metrics? This was a very serious bug for us, causing non-trivial data loss as our messages have a QoS timing window for prompt delivery that we missed. If an engineer has not manually noticed that a reported metric was increasing over time, this could have gone on for much longer than a week. If we had not had a stale message detector, we could have reprocessed and resent messages we sent multiple days ago (that our standard pulsar idempotency detector would have missed). The fact that this issue was manually discovered by a lucky glance as a dashboard and not due to failing health checks or other standard metrics also has us concerned about the suitability for our high performance workload. ### Error messages ```text java.util.NoSuchElementException: null at it.unimi.dsi.fastutil.longs.Long2ObjectAVLTreeMap.firstLongKey(Long2ObjectAVLTreeMap.java:934) ~[it.unimi.dsi-fastutil-8.5.16.jar:?] at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.nextDeliveryTime(InMemoryDelayedDeliveryTracker.java:273) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2] at org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker.updateTimer(AbstractDelayedDeliveryTracker.java:99) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2] at org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTracker.getScheduledMessages(InMemoryDelayedDeliveryTracker.java:229) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.getMessagesToReplayNow(PersistentDispatcherMultipleConsumers.java:1327) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readMoreEntries(PersistentDispatcherMultipleConsumers.java:385) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2] at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.lambda$readMoreEntriesAsync$4(PersistentDispatcherMultipleConsumers.java:324) ~[org.apache.pulsar-pulsar-broker-4.1.2.jar:4.1.2] at io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173) ~[io.netty-netty-common-4.1.128.Final.jar:4.1.128.Final] at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166) ~[io.netty-netty-common-4.1.128.Final.jar:4.1.128.Final] at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472) ~[io.netty-netty-common-4.1.128.Final.jar:4.1.128.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:405) ~[io.netty-netty-transport-classes-epoll-4.1.128.Final.jar:4.1.128.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:998) ~[io.netty-netty-common-4.1.128.Final.jar:4.1.128.Final] at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) ~[io.netty-netty-common-4.1.128.Final.jar:4.1.128.Final] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.128.Final.jar:4.1.128.Final] at java.base/java.lang.Thread.run(Unknown Source) [?:?] ``` ### Reproducing the issue We are attempting to reproduce this in a synthetic testing environments and will update the issue if we can come up with one. Our average write and process speed is on the order of 1.5k / second on roughly 20 topics. This feature would write a much smaller amount (order 10/second) with short but intense spikes of 4k / second or more with deliver_after_time that are very similar or identical times set. ### Additional information I found https://github.com/apache/pulsar/issues/25174 which has some of the same hallmarks: - A broker exception - That seems to cause scheduled messages to not be dispatched - That is resolved by restarting the broker However, the stack trace is different. ### 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]
