|
19 | 19 | package org.apache.bookkeeper.mledger.impl; |
20 | 20 |
|
21 | 21 | import static java.nio.charset.StandardCharsets.UTF_8; |
| 22 | +import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER; |
22 | 23 | import static org.testng.Assert.assertEquals; |
23 | 24 | import static org.testng.Assert.assertFalse; |
24 | 25 | import static org.testng.Assert.assertNotEquals; |
|
50 | 51 | import org.apache.bookkeeper.mledger.Position; |
51 | 52 | import org.apache.bookkeeper.mledger.PositionFactory; |
52 | 53 | import org.apache.bookkeeper.test.MockedBookKeeperTestCase; |
| 54 | +import org.apache.commons.lang3.tuple.Pair; |
| 55 | +import org.apache.pulsar.common.api.proto.CommandSubscribe; |
| 56 | +import org.awaitility.Awaitility; |
| 57 | +import org.mockito.Mockito; |
| 58 | +import org.mockito.stubbing.Answer; |
53 | 59 | import org.slf4j.Logger; |
54 | 60 | import org.slf4j.LoggerFactory; |
55 | 61 | import org.testng.Assert; |
@@ -103,6 +109,58 @@ void testOpenNonDurableCursorAtNonExistentMessageId() throws Exception { |
103 | 109 | ledger.close(); |
104 | 110 | } |
105 | 111 |
|
| 112 | + @Test(timeOut = 20000) |
| 113 | + void testOpenNonDurableCursorWhileLedgerIsAddingFirstEntryAfterTrimmed() throws Exception { |
| 114 | + ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(1) |
| 115 | + .setRetentionTime(0, TimeUnit.MILLISECONDS); |
| 116 | + config.setMinimumRolloverTime(0, TimeUnit.MILLISECONDS); |
| 117 | + @Cleanup |
| 118 | + ManagedLedgerImpl ledgerSpy = |
| 119 | + Mockito.spy((ManagedLedgerImpl) factory.open("non_durable_cursor_while_ledger_trimmed", config)); |
| 120 | + |
| 121 | + ledgerSpy.addEntry("message1".getBytes()); |
| 122 | + |
| 123 | + ledgerSpy.rollCurrentLedgerIfFull(); |
| 124 | + Awaitility.waitAtMost(5, TimeUnit.SECONDS).until(() -> |
| 125 | + ledgerSpy.getLedgersInfoAsList().size() > 1 |
| 126 | + ); |
| 127 | + CompletableFuture<Void> trimFuture = new CompletableFuture<>(); |
| 128 | + ledgerSpy.trimConsumedLedgersInBackground(trimFuture); |
| 129 | + trimFuture.join(); |
| 130 | + |
| 131 | + // Use (currentLedgerId, -1) as startCursorPosition after ledger was trimmed |
| 132 | + Position startCursorPosition = PositionFactory.create(ledgerSpy.getCurrentLedger().getId(), -1); |
| 133 | + assertTrue(startCursorPosition.compareTo(ledgerSpy.lastConfirmedEntry) > 0); |
| 134 | + |
| 135 | + CountDownLatch getLastPositionLatch = new CountDownLatch(1); |
| 136 | + CountDownLatch newNonDurableCursorLatch = new CountDownLatch(1); |
| 137 | + Mockito.when(ledgerSpy.getLastPositionAndCounter()).then((Answer<Pair<Position, Long>>) invocation -> { |
| 138 | + newNonDurableCursorLatch.countDown(); |
| 139 | + getLastPositionLatch.await(); |
| 140 | + return Pair.of(ledgerSpy.lastConfirmedEntry, ENTRIES_ADDED_COUNTER_UPDATER.get(ledgerSpy)); |
| 141 | + }); |
| 142 | + |
| 143 | + CompletableFuture<ManagedCursor> cursorFuture = new CompletableFuture<ManagedCursor>() |
| 144 | + .completeAsync(() -> |
| 145 | + new NonDurableCursorImpl(bkc, ledgerSpy, "my_test_cursor", |
| 146 | + startCursorPosition, CommandSubscribe.InitialPosition.Latest, false) |
| 147 | + ); |
| 148 | + Position oldLastConfirmedEntry = ledgerSpy.lastConfirmedEntry; |
| 149 | + |
| 150 | + // Wait until NonDurableCursorImpl constructor invokes ManagedLedgerImpl.getLastPositionAndCounter |
| 151 | + newNonDurableCursorLatch.await(); |
| 152 | + // Add first entry after ledger was trimmed |
| 153 | + ledgerSpy.addEntry("message2".getBytes()); |
| 154 | + assertTrue(oldLastConfirmedEntry.compareTo(ledgerSpy.lastConfirmedEntry) < 0); |
| 155 | + |
| 156 | + // Unblock NonDurableCursorImpl constructor |
| 157 | + getLastPositionLatch.countDown(); |
| 158 | + |
| 159 | + // cursor should read from lastConfirmedEntry |
| 160 | + ManagedCursor cursor = cursorFuture.join(); |
| 161 | + assertEquals(cursor.getReadPosition(), ledgerSpy.lastConfirmedEntry); |
| 162 | + } |
| 163 | + |
106 | 164 | @Test(timeOut = 20000) |
107 | 165 | void testZNodeBypassed() throws Exception { |
108 | 166 | ManagedLedger ledger = factory.open("my_test_ledger"); |
|
0 commit comments