Shawyeok opened a new issue, #25174: URL: https://github.com/apache/pulsar/issues/25174
### 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 3.0.14 ### Issue Description ### **Issue Summary** After upgrading to Pulsar 3.0.14, we observed `NoSuchElementException` errors in the broker logs that cause the dispatcher to stop dispatching messages to consumers. ### **Error Log** ``` 2026-01-19T07:50:56.200Z [broker-topic-workers-OrderedExecutor-10-0] ERROR org.apache.bookkeeper.common.util.SingleThreadExecutor - Error while running task: null java.util.NoSuchElementException at java.base/java.util.concurrent.ConcurrentSkipListMap.firstKey(ConcurrentSkipListMap.java:1859) at org.apache.bookkeeper.mledger.impl.EntryCountEstimator.internalEstimateEntryCountByBytesSize(EntryCountEstimator.java:93) at org.apache.bookkeeper.mledger.impl.EntryCountEstimator.estimateEntryCountByBytesSize(EntryCountEstimator.java:51) at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.applyMaxSizeCap(ManagedCursorImpl.java:3849) at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntriesWithSkipOrWait(ManagedCursorImpl.java:1051) at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.asyncReadEntriesOrWait(ManagedCursorImpl.java:1031) at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readMoreEntries(PersistentDispatcherMultipleConsumers.java:371) at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.lambda$readEntriesComplete$8(PersistentDispatcherMultipleConsumers.java:628) at org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137) at org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.base/java.lang.Thread.run(Thread.java:840) ``` ### **Root Cause Analysis** The bug is in `EntryCountEstimator.internalEstimateEntryCountByBytesSize()` at line 93: ```java } else if (readPosition.getLedgerId() < ledgersInfo.firstKey()) { readPosition = PositionImpl.get(ledgersInfo.firstKey(), 0); } ``` The code calls `ledgersInfo.firstKey()` on a `ConcurrentSkipListMap` without first checking if the map is empty. When `ledgersInfo` is empty (which can occur due to race conditions during ledger trimming/compaction), `firstKey()` throws `NoSuchElementException`. Similarly, line 89 also has a potential issue: ```java } else if (lastLedgerId == null && readPosition.getLedgerId() > ledgersInfo.lastKey()) { ``` Maybe safer alternatives like `firstEntry()` with null checks should be used. ### **Impact** - The uncaught exception causes the dispatcher to stop dispatching messages to consumers - The subscription appears stuck with `subscriptionHavePendingRead: true` but `waitingReadOp: false` - Consumers will not receive any new messages until the broker/topic is restarted/unloaded. ### **Managed Ledger Internal Stats** This shows the state of an affected topic. Note that the cursor's `readPosition` (20144233:0) references a ledger that no longer exists in the ledgers list (ledgers start from 20223153): ```json { "entriesAddedCounter": 3, "numberOfEntries": 3, "totalSize": 1598, "lastConfirmedEntry": "20227219:0", "state": "LedgerOpened", "ledgers": [ {"ledgerId": 20223153, "entries": 1, "size": 533}, {"ledgerId": 20227219, "entries": 1, "size": 532}, {"ledgerId": 20251815, "entries": 0, "size": 0} ], "cursors": { "<subname>": { "markDeletePosition": "19058261:1", "readPosition": "20144233:0", "waitingReadOp": false, "subscriptionHavePendingRead": true, "subscriptionHavePendingReplayRead": false, "active": true } } } ``` ### Error messages ```text ``` ### Reproducing the issue I don't have a stable reproducer yet. ### Additional information _No response_ ### 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]
