|
20 | 20 |
|
21 | 21 | import static org.apache.pulsar.common.util.PortManager.releaseLockedPort; |
22 | 22 | import static org.testng.Assert.assertEquals; |
| 23 | +import static org.testng.Assert.assertNotEquals; |
23 | 24 | import static org.testng.Assert.assertNotNull; |
24 | 25 | import static org.testng.Assert.assertTrue; |
25 | 26 | import static org.testng.Assert.fail; |
26 | 27 |
|
| 28 | +import com.fasterxml.jackson.databind.ObjectMapper; |
27 | 29 | import java.nio.charset.StandardCharsets; |
28 | 30 | import java.util.ArrayList; |
29 | 31 | import java.util.List; |
| 32 | +import java.util.UUID; |
30 | 33 | import java.util.concurrent.CountDownLatch; |
31 | 34 | import java.util.concurrent.CyclicBarrier; |
32 | 35 | import java.util.concurrent.Future; |
33 | 36 | import java.util.concurrent.TimeUnit; |
34 | 37 | import java.util.concurrent.atomic.AtomicBoolean; |
35 | 38 | import java.util.concurrent.atomic.AtomicReference; |
36 | 39 |
|
| 40 | +import lombok.extern.slf4j.Slf4j; |
37 | 41 | import org.apache.bookkeeper.client.BookKeeper; |
38 | 42 | import org.apache.bookkeeper.client.BookKeeperTestClient; |
| 43 | +import org.apache.bookkeeper.client.LedgerEntry; |
39 | 44 | import org.apache.bookkeeper.client.api.DigestType; |
40 | 45 | import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; |
41 | 46 | import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; |
|
49 | 54 | import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; |
50 | 55 | import org.apache.bookkeeper.mledger.Position; |
51 | 56 | import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; |
| 57 | +import org.apache.bookkeeper.mledger.proto.MLDataFormats; |
52 | 58 | import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; |
53 | 59 | import org.apache.bookkeeper.test.BookKeeperClusterTestCase; |
54 | 60 | import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; |
55 | | -import org.apache.pulsar.common.util.collections.LongPairRangeSet; |
| 61 | +import org.awaitility.Awaitility; |
| 62 | +import org.testng.annotations.DataProvider; |
56 | 63 | import org.awaitility.Awaitility; |
57 | 64 | import org.testng.annotations.Test; |
58 | 65 |
|
59 | 66 | import io.netty.buffer.ByteBuf; |
60 | 67 | import lombok.Cleanup; |
61 | 68 |
|
| 69 | +@Slf4j |
62 | 70 | public class ManagedLedgerBkTest extends BookKeeperClusterTestCase { |
63 | 71 |
|
| 72 | + private final ObjectMapper jackson = new ObjectMapper(); |
| 73 | + |
64 | 74 | public ManagedLedgerBkTest() { |
65 | 75 | super(2); |
66 | 76 | } |
@@ -590,44 +600,114 @@ public void testPeriodicRollover() throws Exception { |
590 | 600 | Awaitility.await().until(() -> cursorImpl.getCursorLedger() != currentLedgerId); |
591 | 601 | } |
592 | 602 |
|
| 603 | + @DataProvider(name = "unackedRangesOpenCacheSetEnabledPair") |
| 604 | + public Object[][] unackedRangesOpenCacheSetEnabledPair() { |
| 605 | + return new Object[][]{ |
| 606 | + {false, true}, |
| 607 | + {true, false}, |
| 608 | + {true, true}, |
| 609 | + {false, false} |
| 610 | + }; |
| 611 | + } |
| 612 | + |
593 | 613 | /** |
594 | 614 | * This test validates that cursor serializes and deserializes individual-ack list from the bk-ledger. |
595 | | - * |
596 | 615 | * @throws Exception |
597 | 616 | */ |
598 | | - @Test |
599 | | - public void testUnackmessagesAndRecovery() throws Exception { |
| 617 | + @Test(dataProvider = "unackedRangesOpenCacheSetEnabledPair") |
| 618 | + public void testUnackmessagesAndRecoveryCompatibility(boolean enabled1, boolean enabled2) throws Exception { |
| 619 | + final String mlName = "ml" + UUID.randomUUID().toString().replaceAll("-", ""); |
| 620 | + final String cursorName = "c1"; |
600 | 621 | ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); |
601 | | - factoryConf.setMaxCacheSize(0); |
602 | | - |
603 | 622 | ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); |
604 | | - |
605 | | - ManagedLedgerConfig config = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) |
| 623 | + final ManagedLedgerConfig config1 = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) |
| 624 | + .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) |
| 625 | + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1) |
| 626 | + .setUnackedRangesOpenCacheSetEnabled(enabled1); |
| 627 | + final ManagedLedgerConfig config2 = new ManagedLedgerConfig().setEnsembleSize(1).setWriteQuorumSize(1) |
606 | 628 | .setAckQuorumSize(1).setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1) |
607 | | - .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1); |
608 | | - ManagedLedger ledger = factory.open("my_test_unack_messages", config); |
609 | | - ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("c1"); |
| 629 | + .setMaxUnackedRangesToPersistInMetadataStore(1).setMaxEntriesPerLedger(5).setMetadataAckQuorumSize(1) |
| 630 | + .setUnackedRangesOpenCacheSetEnabled(enabled2); |
| 631 | + |
| 632 | + ManagedLedger ledger1 = factory.open(mlName, config1); |
| 633 | + ManagedCursorImpl cursor1 = (ManagedCursorImpl) ledger1.openCursor(cursorName); |
610 | 634 |
|
611 | 635 | int totalEntries = 100; |
612 | 636 | for (int i = 0; i < totalEntries; i++) { |
613 | | - Position p = ledger.addEntry("entry".getBytes()); |
| 637 | + Position p = ledger1.addEntry("entry".getBytes()); |
614 | 638 | if (i % 2 == 0) { |
615 | | - cursor.delete(p); |
| 639 | + cursor1.delete(p); |
616 | 640 | } |
617 | 641 | } |
| 642 | + log.info("ack ranges: {}", cursor1.getIndividuallyDeletedMessagesSet().size()); |
618 | 643 |
|
619 | | - LongPairRangeSet<PositionImpl> unackMessagesBefore = cursor.getIndividuallyDeletedMessagesSet(); |
| 644 | + // reopen and recover cursor |
| 645 | + ledger1.close(); |
| 646 | + ManagedLedger ledger2 = factory.open(mlName, config2); |
| 647 | + ManagedCursorImpl cursor2 = (ManagedCursorImpl) ledger2.openCursor(cursorName); |
620 | 648 |
|
621 | | - ledger.close(); |
| 649 | + log.info("before: {}", cursor1.getIndividuallyDeletedMessagesSet().asRanges()); |
| 650 | + log.info("after : {}", cursor2.getIndividuallyDeletedMessagesSet().asRanges()); |
| 651 | + assertEquals(cursor1.getIndividuallyDeletedMessagesSet().asRanges(), cursor2.getIndividuallyDeletedMessagesSet().asRanges()); |
| 652 | + assertEquals(cursor1.markDeletePosition, cursor2.markDeletePosition); |
622 | 653 |
|
623 | | - // open and recover cursor |
624 | | - ledger = factory.open("my_test_unack_messages", config); |
625 | | - cursor = (ManagedCursorImpl) ledger.openCursor("c1"); |
| 654 | + ledger2.close(); |
| 655 | + factory.shutdown(); |
| 656 | + } |
626 | 657 |
|
627 | | - LongPairRangeSet<PositionImpl> unackMessagesAfter = cursor.getIndividuallyDeletedMessagesSet(); |
628 | | - assertTrue(unackMessagesBefore.equals(unackMessagesAfter)); |
| 658 | + @DataProvider(name = "booleans") |
| 659 | + public Object[][] booleans() { |
| 660 | + return new Object[][] { |
| 661 | + {true}, |
| 662 | + {false}, |
| 663 | + }; |
| 664 | + } |
629 | 665 |
|
630 | | - ledger.close(); |
| 666 | + @Test(dataProvider = "booleans") |
| 667 | + public void testConfigPersistIndividualAckAsLongArray(boolean enable) throws Exception { |
| 668 | + final String mlName = "ml" + UUID.randomUUID().toString().replaceAll("-", ""); |
| 669 | + final String cursorName = "c1"; |
| 670 | + ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); |
| 671 | + ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, factoryConf); |
| 672 | + final ManagedLedgerConfig config = new ManagedLedgerConfig() |
| 673 | + .setEnsembleSize(1).setWriteQuorumSize(1).setAckQuorumSize(1) |
| 674 | + .setMetadataEnsembleSize(1).setMetadataWriteQuorumSize(1).setMetadataAckQuorumSize(1) |
| 675 | + .setMaxUnackedRangesToPersistInMetadataStore(1) |
| 676 | + .setUnackedRangesOpenCacheSetEnabled(true).setPersistIndividualAckAsLongArray(enable); |
| 677 | + |
| 678 | + ManagedLedger ledger1 = factory.open(mlName, config); |
| 679 | + ManagedCursorImpl cursor1 = (ManagedCursorImpl) ledger1.openCursor(cursorName); |
| 680 | + |
| 681 | + // Write entries. |
| 682 | + int totalEntries = 100; |
| 683 | + List<Position> entries = new ArrayList<>(); |
| 684 | + for (int i = 0; i < totalEntries; i++) { |
| 685 | + Position p = ledger1.addEntry("entry".getBytes()); |
| 686 | + entries.add(p); |
| 687 | + } |
| 688 | + // Make ack holes and trigger a mark deletion. |
| 689 | + for (int i = totalEntries - 1; i >=0 ; i--) { |
| 690 | + if (i % 2 == 0) { |
| 691 | + cursor1.delete(entries.get(i)); |
| 692 | + } |
| 693 | + } |
| 694 | + cursor1.markDelete(entries.get(9)); |
| 695 | + Awaitility.await().untilAsserted(() -> { |
| 696 | + assertEquals(cursor1.pendingMarkDeleteOps.size(), 0); |
| 697 | + }); |
| 698 | + |
| 699 | + // Verify: the config affects. |
| 700 | + long cursorLedgerLac = cursor1.cursorLedger.getLastAddConfirmed(); |
| 701 | + LedgerEntry ledgerEntry = cursor1.cursorLedger.readEntries(cursorLedgerLac, cursorLedgerLac).nextElement(); |
| 702 | + MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); |
| 703 | + if (enable) { |
| 704 | + assertNotEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0); |
| 705 | + } else { |
| 706 | + assertEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0); |
| 707 | + } |
| 708 | + |
| 709 | + // cleanup |
| 710 | + ledger1.close(); |
631 | 711 | factory.shutdown(); |
632 | 712 | } |
633 | 713 | } |
0 commit comments