oneby-wang commented on PR #25144:
URL: https://github.com/apache/pulsar/pull/25144#issuecomment-3752582871
I occasionally found `ManagedLedgerTest.testLockReleaseWhenTrimLedger()`
test was flaky. Here is a full log when the test throws
`ManagedLedgerException`.
```
2026-01-13T11:07:22,410 - INFO - [main:ManagedLedgerImpl@418] - Opening
managed ledger testLockReleaseWhenTrimLedger
2026-01-13T11:07:22,410 - INFO -
[bookkeeper-ml-scheduler-OrderedScheduler-7-0:MetaStoreImpl@113] - Creating
'/managed-ledgers/testLockReleaseWhenTrimLedger'
2026-01-13T11:07:22,415 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 3
2026-01-13T11:07:22,416 - INFO -
[test-OrderedScheduler-1-0:ManagedLedgerImpl@614] -
[testLockReleaseWhenTrimLedger] Created ledger 3 after closed null
2026-01-13T11:07:22,416 - INFO -
[bookkeeper-ml-scheduler-OrderedScheduler-7-0:ManagedLedgerFactoryImpl$2@482] -
[testLockReleaseWhenTrimLedger] Successfully initialize managed ledger
2026-01-13T11:07:22,419 - INFO - [main:ManagedCursorImpl@831] -
[testLockReleaseWhenTrimLedger] Cursor
test-cursor8784046c-0e28-4f04-8c8b-315231fb9b57 recovered to position 3:-1
2026-01-13T11:07:22,426 - INFO -
[bookkeeper-ml-scheduler-OrderedScheduler-7-0:ManagedLedgerImpl$6@1049] -
[testLockReleaseWhenTrimLedger] Opened new cursor:
ManagedCursorImpl{ledger=testLockReleaseWhenTrimLedger,
name=test-cursor8784046c-0e28-4f04-8c8b-315231fb9b57, ackPos=3:-1, readPos=3:0}
2026-01-13T11:07:22,780 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 3 for being full
2026-01-13T11:07:22,808 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 3
2026-01-13T11:07:22,810 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 4
2026-01-13T11:07:22,811 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 4
2026-01-13T11:07:22,842 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 4 for being full
2026-01-13T11:07:22,844 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 4
2026-01-13T11:07:22,845 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 5
2026-01-13T11:07:22,845 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 5
2026-01-13T11:07:23,159 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 5 for being full
2026-01-13T11:07:23,161 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 5
2026-01-13T11:07:23,161 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 6
2026-01-13T11:07:23,162 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 6
2026-01-13T11:07:23,170 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 6 for being full
2026-01-13T11:07:23,171 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 6
2026-01-13T11:07:23,172 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 7
2026-01-13T11:07:23,173 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 7
2026-01-13T11:07:23,249 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 7 for being full
2026-01-13T11:07:23,250 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 7
2026-01-13T11:07:23,251 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 8
2026-01-13T11:07:23,252 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 8
2026-01-13T11:07:23,254 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 8 for being full
2026-01-13T11:07:23,255 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 8
2026-01-13T11:07:23,256 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 9
2026-01-13T11:07:23,257 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 9
2026-01-13T11:07:23,259 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 9 for being full
2026-01-13T11:07:23,260 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 9
2026-01-13T11:07:23,261 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 10
2026-01-13T11:07:23,262 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 10
2026-01-13T11:07:23,265 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 10 for being full
2026-01-13T11:07:23,266 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 10
2026-01-13T11:07:23,272 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 11
2026-01-13T11:07:23,273 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 11
2026-01-13T11:07:23,556 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 11 for being full
2026-01-13T11:07:23,565 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 11
2026-01-13T11:07:23,566 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 12
2026-01-13T11:07:23,567 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 12
2026-01-13T11:07:23,650 - INFO - [test-OrderedScheduler-1-0:OpAddEntry@284]
- [testLockReleaseWhenTrimLedger] Closing ledger 12 for being full
2026-01-13T11:07:23,651 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1976] -
[testLockReleaseWhenTrimLedger] Creating a new ledger after closed 12
2026-01-13T11:07:23,748 - INFO -
[test-OrderedScheduler-0-0:PulsarMockBookKeeper@138] - Creating ledger 13
2026-01-13T11:07:23,801 - INFO -
[test-OrderedScheduler-0-0:ManagedLedgerImpl@1745] -
[testLockReleaseWhenTrimLedger] Created new ledger 13
2026-01-13T11:07:24,782 - WARN -
[test-OrderedScheduler-1-0:OpReadEntry@168] -
[testLockReleaseWhenTrimLedger][test-cursor8784046c-0e28-4f04-8c8b-315231fb9b57]
read failed from ledger at position:3:0
org.apache.bookkeeper.mledger.ManagedLedgerException: LastConfirmedEntry is
3:-1 when reading entry 0
org.apache.bookkeeper.mledger.ManagedLedgerException: LastConfirmedEntry is
3:-1 when reading entry 0
```
I was every confused since
`ledger.addEntry(String.valueOf(i).getBytes(Encoding))` is an sync method. The
`lastConfirmedEntry` should not be `3:-1`(which is the initial value) when
executing `List<Entry> entryList = cursor.readEntries(entries)`.
https://github.com/apache/pulsar/blob/1fcdf8bb9a97fbc4c0db78c4aa8df2b1fae02a65/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java#L39-L51
The `entryCache` is initialized using `ManagedLedgerImpl.this`.
https://github.com/apache/pulsar/blob/1fcdf8bb9a97fbc4c0db78c4aa8df2b1fae02a65/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L369-L396
The only inconsistent cause in code is here, we returns old
`currentEntryCache` and ignore new `ml` in `newEntryCache`.
https://github.com/apache/pulsar/blob/1fcdf8bb9a97fbc4c0db78c4aa8df2b1fae02a65/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheManagerImpl.java#L87-L102
I searched all the call chains to this method, and came out to a reason:
there should be exactly one invocation of the
`RangeEntryCacheManagerImpl.getEntryCache()` method. So I add a warn log in `if
(currentEntryCache != null)` block(see PR code change), and found no warn log
printed when `ManagedLedgerException` throwed.
Finally, I found that the root cause of the issue was the interaction
between the spy and volatile fields. I'll explain in the following comments.
--
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]