You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
The test doesn't wait for new ledger created, so assertEquals(ledger.ledgers.size() - 1, entries) or assertEquals(ledger.ledgerCache.size() - 1, entries - 1) may be flaky, see code comments.
java.lang.AssertionError:
Expected :10
Actual :9
<Click to see difference>
at org.testng.Assert.fail(Assert.java:110)
at org.testng.Assert.failNotEquals(Assert.java:1577)
at org.testng.Assert.assertEqualsImpl(Assert.java:149)
at org.testng.Assert.assertEquals(Assert.java:131)
at org.testng.Assert.assertEquals(Assert.java:1418)
at org.testng.Assert.assertEquals(Assert.java:1382)
at org.testng.Assert.assertEquals(Assert.java:1428)
at org.apache.bookkeeper.mledger.impl.ManagedLedgerTest.testLockReleaseWhenTrimLedger(ManagedLedgerTest.java:3982)
The test spies an instance that contains volatile fields, which also cause flakiness, see PR comments below.
Modifications
Waiting for new ledger created to avoid flaky test.
Use overriding instead of spying to avoid flaky test.
Verifying this change
Make sure that the change passes the CI checks.
If the box was checked, please highlight the changes
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).
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.
I am not familiar with mockito or bytecode manipulation technology, but after some deeper debug on this test, I'd suggest that we should avoid spying an instance that contains volatile fields in tests, especially when these volatile fields are not accessed using setter and getter methods.
Here is the debug details:
All fields of originalLedger and ledger are identical after initialization.
lastConfirmedEntry is also identical.
Except entryCahce.ml field:
entryCahce.ml is originalLedger in originalLedger.
entryCahce.ml is originalLedger in ledger.
This is the inconsistency root cause.
After adding 10 entries, lastConfirmedEntry in ledger is updated to 12:0, but in originalLedger, lastConfirmedEntry is still 3:-1, which is not updated.
After adding 10 entries, currentLedger in originalLedger is also not updated.
The flaky test was very hard to reproduce, but I occasionally found a way to easily reproduce it by adding some breakpoints.
I don't know the reason, maybe breakpoints in IntelliJ IDEA cause further de-synchronization between the volatile fields of the spy object and the original object, and it seems that breakpoints added after ledger.addEntry(String.valueOf(i).getBytes(Encoding)) method are more likely to cause flakiness.
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Motivation
Fix
ManagedLedgerTest.testLockReleaseWhenTrimLedger()flaky test.This test has the following problems:
assertEquals(ledger.ledgers.size() - 1, entries)orassertEquals(ledger.ledgerCache.size() - 1, entries - 1)may be flaky, see code comments.Modifications
Verifying this change
If the box was checked, please highlight the changes
Documentation
docdoc-requireddoc-not-neededdoc-completeMatching PR in forked repository
PR in forked repository: oneby-wang#24