Skip to content

Commit

Permalink
Fix the flaky test in the ManagedLedgerTest (#11016)
Browse files Browse the repository at this point in the history
fixes testExpiredLedgerDeletionAfterManagedLedgerRestart

(cherry picked from commit 1623790)
  • Loading branch information
codelipenghui authored and eolivelli committed Jun 22, 2021
1 parent 02ec937 commit a9b49ef
Showing 1 changed file with 12 additions and 10 deletions.
Expand Up @@ -108,6 +108,7 @@
import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.metadata.api.Stat;
import org.awaitility.Awaitility;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
Expand Down Expand Up @@ -2940,11 +2941,6 @@ public void testExpiredLedgerDeletionAfterManagedLedgerRestart() throws Exceptio
Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), 2);
List<Entry> entries = cursor.readEntries(3);

for (Entry entry : entries) {
cursor.markDelete(entry.getPosition());
}
entries.forEach(e -> e.release());

// managed-ledger restart
managedLedger.close();
managedLedger = (ManagedLedgerImpl) factory.open("ml_restart_ledger", config);
Expand All @@ -2953,14 +2949,20 @@ public void testExpiredLedgerDeletionAfterManagedLedgerRestart() throws Exceptio
// and now ledgers are [{entries=2}, {entries=1}, {entries=0}]
Assert.assertTrue(managedLedger.getLedgersInfoAsList().size() >= 2);

cursor = managedLedger.openCursor("c1");
for (Entry entry : entries) {
cursor.markDelete(entry.getPosition());
}
entries.forEach(Entry::release);
// Now we update the cursors that are still subscribing to ledgers that has been consumed completely
managedLedger.maybeUpdateCursorBeforeTrimmingConsumedLedger();
managedLedger.internalTrimConsumedLedgers(Futures.NULL_PROMISE);
Thread.sleep(100);

// We only have one empty ledger at last [{entries=0}]
Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), 1);
Assert.assertEquals(managedLedger.getTotalSize(), 0);
ManagedLedgerImpl finalManagedLedger = managedLedger;
Awaitility.await().untilAsserted(() -> {
// We only have one empty ledger at last [{entries=0}]
Assert.assertEquals(finalManagedLedger.getLedgersInfoAsList().size(), 1);
Assert.assertEquals(finalManagedLedger.getTotalSize(), 0);
});
}

@Test(timeOut = 20000)
Expand Down

0 comments on commit a9b49ef

Please sign in to comment.