-
Notifications
You must be signed in to change notification settings - Fork 3.7k
Description
Search before reporting
- I searched in the issues and found nothing similar.
Read release policy
- I understand that unsupported 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:
} 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:
} 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: truebutwaitingReadOp: 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):
{
"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
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!