BewareMyPower opened a new issue, #25118: URL: https://github.com/apache/pulsar/issues/25118
### 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 It happens on Pulsar 4.0.5, but this bug also exists on master branch. ### Issue Description There is a topic (`persistent://public/__kafka/__consumer_offsets-partition-3`) whose namespace retention policy has 3 days time limit and no size limit. It has 200+ ledgers. One day, we noticed the mark-delete position of the `__compaction` cursor was reset to a very old position that is not included in all current ledgers: ```json "numberOfEntries" : 21782357, "lastConfirmedEntry" : "2306104:0", "state" : "LedgerOpened", "ledgers" : [ { "ledgerId" : 2276367, "entries" : 50000, "size" : 17739504, "offloaded" : false, "underReplicated" : false }, { ``` ```json "__compaction" : { "markDeletePosition" : "1877:52", "readPosition" : "2281596:35302", ``` where ledger 2281596 is the 75th ledger of all 200+ ledgers. ### Error messages ```text We noticed the cursor's mark-delete position was reset to an old position during compaction phase two when the broker was closing. Key logs: 2025-12-09T07:17:14,466+0000 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO org.apache.pulsar.compaction.AbstractTwoPhaseCompactor - Commencing phase two of compaction for persistent://public/__kafka/__consumer_offsets-partition-3, from 1877:53:3:-1 to 2304038:5078:3:-1, compacting 71 keys to ledger 2304479 2025-12-09T07:17:14,484+0000 [BookKeeperClientWorker-OrderedExecutor-1-0] INFO org.apache.bookkeeper.mledger.impl.ManagedCursorImpl - [public/__kafka/persistent/__consumer_offsets-partition-3] reset readPosition to 1877:53 (ackSet is null) before current read readPosition 2304038:5079 on cursor __compaction 2025-12-09T07:17:14,485+0000 [pulsar-io-9-3] INFO org.apache.pulsar.client.impl.ConsumerImpl - [persistent://public/__kafka/__consumer_offsets-partition-3][__compaction] Successfully reset subscription to the message 1877:53:3:-1 2025-12-09T07:17:18,045+0000 [pulsar-service-shutdown] INFO org.apache.pulsar.broker.service.BrokerService - [persistent://public/__kafka/__consumer_offsets-partition-3] Unloading topic 2025-12-09T07:17:18,047+0000 [pulsar-service-shutdown] INFO org.apache.pulsar.broker.service.persistent.PersistentSubscription - [persistent://public/__kafka/__consumer_offsets-partition-3][__compaction] Successfully closed subscription [ManagedCursorImpl{ledger=public/__kafka/persistent/__consumer_offsets-partition-3, name=__compaction, ackPos=1877:52, readPos=269017:14425}] 2025-12-09T07:17:18,049+0000 [pulsar-service-shutdown] INFO org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/__kafka/persistent/__consumer_offsets-partition-3] Closing managed ledger 025-12-09T07:17:18,198+0000 [broker-client-shared-internal-executor-11-1] WARN org.apache.pulsar.broker.service.persistent.PersistentTopic - [persistent://public/__kafka/__con sumer_offsets-partition-3] Compaction failure. java.util.concurrent.CompletionException: java.util.concurrent.CancellationException at java.base/java.util.concurrent.CompletableFuture.encodeThrowable(Unknown Source) ~[?:?] at java.base/java.util.concurrent.CompletableFuture.completeThrowable(Unknown Source) ~[?:?] at java.base/java.util.concurrent.CompletableFuture$UniAccept.tryFire(Unknown Source) ~[?:?] at java.base/java.util.concurrent.CompletableFuture.postComplete(Unknown Source) ~[?:?] at java.base/java.util.concurrent.CompletableFuture.cancel(Unknown Source) ~[?:?] at org.apache.pulsar.client.impl.RawReaderImpl$RawConsumerImpl.lambda$failPendingRawReceives$1(RawReaderImpl.java:213) ``` ### Reproducing the issue I will include the reproduce code in a PR. ### Additional information The root cause is that during compaction phase two, the reader will seek to the 1st compacted entry's position to read the whole topic. If topic was closing, then the pending `readAsync` future would fail with `CancellationException` by `CompletableFuture#cancel`. Then the cursor will update the mark-delete position with the 1st compacted entry's position, which is incorrect. For example, given the following original ledgers: - 0: 0 to 9 - 1: 0 to 9 - 2: 0 and compacted entries: (0, 5), (1,9), horizon is 1:9, (2,0) is not compacted. Then in phase two, the mark-delete position will be set to (0, 5). Next time the phase one will read the original ledgers from (0, 5) to (2, 0). ### 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]
