ledger.offloadPrefix(PositionImpl.earliest); Assert.fail("Should have thrown an exception"); } catch (ManagedLedgerException.InvalidCursorPositionException e) { ledger.offloadPrefix(PositionImpl.latest); Assert.fail("Should have thrown an exception"); } catch (ManagedLedgerException.InvalidCursorPositionException e) {
@Test public void testOffloadSamePositionTwice() throws Exception { MockLedgerOffloader offloader = new MockLedgerOffloader(); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(10); config.setMinimumRolloverTime(0, TimeUnit.SECONDS); config.setRetentionTime(10, TimeUnit.MINUTES); config.setLedgerOffloader(offloader); ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config); int i = 0; for (; i < 25; i++) { String content = "entry-" + i; ledger.addEntry(content.getBytes()); } Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); ledger.offloadPrefix(ledger.getLastConfirmedEntry()); Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getOffloadContext().getComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); ledger.offloadPrefix(ledger.getLastConfirmedEntry()); Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getOffloadContext().getComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); }
@Test public void testOffloadNewML() throws Exception { MockLedgerOffloader offloader = new MockLedgerOffloader(); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(10); config.setMinimumRolloverTime(0, TimeUnit.SECONDS); config.setRetentionTime(10, TimeUnit.MINUTES); config.setLedgerOffloader(offloader); ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config); try { ledger.offloadPrefix(ledger.getLastConfirmedEntry()); } catch (ManagedLedgerException.InvalidCursorPositionException e) { // expected } // add one entry and try again ledger.addEntry("foobar".getBytes()); Position p = ledger.getLastConfirmedEntry(); Assert.assertEquals(p, ledger.offloadPrefix(ledger.getLastConfirmedEntry())); Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 1); Assert.assertEquals(offloader.offloadedLedgers().size(), 0); }
@Test public void testOffload() throws Exception { MockLedgerOffloader offloader = new MockLedgerOffloader(); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(10); config.setMinimumRolloverTime(0, TimeUnit.SECONDS); config.setRetentionTime(10, TimeUnit.MINUTES); config.setLedgerOffloader(offloader); ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config); int i = 0; for (; i < 25; i++) { String content = "entry-" + i; ledger.addEntry(content.getBytes()); } Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); ledger.offloadPrefix(ledger.getLastConfirmedEntry()); Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getOffloadContext().getComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); }
.filter(e -> e.getOffloadContext().getComplete()).count(), 0); try { ledger.offloadPrefix(p); Assert.fail("Should have thrown an exception"); } catch (ManagedLedgerException e) {
ledger.offloadPrefix(ledger.getLastConfirmedEntry()); } catch (ManagedLedgerException e) { Assert.assertEquals(e.getCause().getClass(), CompletionException.class);
@Test public void testOffloadClosedManagedLedger() throws Exception { MockLedgerOffloader offloader = new MockLedgerOffloader(); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(10); config.setMinimumRolloverTime(0, TimeUnit.SECONDS); config.setRetentionTime(10, TimeUnit.MINUTES); config.setLedgerOffloader(offloader); ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config); for (int i = 0; i < 21; i++) { String content = "entry-" + i; ledger.addEntry(content.getBytes()); } Position p = ledger.getLastConfirmedEntry(); ledger.close(); try { ledger.offloadPrefix(p); Assert.fail("Should fail because ML is closed"); } catch (ManagedLedgerException.ManagedLedgerAlreadyClosedException e) { // expected } Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getOffloadContext().getComplete()).count(), 0); Assert.assertEquals(offloader.offloadedLedgers().size(), 0); }
Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); PositionImpl firstUnoffloaded = (PositionImpl)ledger.offloadPrefix(p); PositionImpl firstUnoffloaded2 = (PositionImpl)ledger.offloadPrefix(ledger.getLastConfirmedEntry()); Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 3); Assert.assertEquals(offloader.offloadedLedgers().size(), 2);
ledgers.get(secondLedgerId).toBuilder().setEntries(0).setSize(0).build()); PositionImpl firstUnoffloaded = (PositionImpl)ledger.offloadPrefix(ledger.getLastConfirmedEntry()); Assert.assertEquals(firstUnoffloaded.getLedgerId(), fourthLedgerId); Assert.assertEquals(firstUnoffloaded.getEntryId(), 0);
ledger.offloadPrefix(ledger.getLastConfirmedEntry()); } catch (ManagedLedgerException e) { ledger.offloadPrefix(ledger.getLastConfirmedEntry());
@Test public void testOffloadDelete() throws Exception { Set<Pair<Long, UUID>> deleted = ConcurrentHashMap.newKeySet(); CompletableFuture<Set<Long>> errorLedgers = new CompletableFuture<>(); Set<Pair<Long, UUID>> failedOffloads = ConcurrentHashMap.newKeySet(); MockLedgerOffloader offloader = new MockLedgerOffloader(); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(10); config.setMinimumRolloverTime(0, TimeUnit.SECONDS); config.setRetentionTime(0, TimeUnit.MINUTES); config.setLedgerOffloader(offloader); ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger", config); ManagedCursor cursor = ledger.openCursor("foobar"); for (int i = 0; i < 15; i++) { String content = "entry-" + i; ledger.addEntry(content.getBytes()); } Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2); ledger.offloadPrefix(ledger.getLastConfirmedEntry()); Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getOffloadContext().getComplete()).count(), 1); Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); long firstLedger = ledger.getLedgersInfoAsList().get(0).getLedgerId(); long secondLedger = ledger.getLedgersInfoAsList().get(1).getLedgerId(); cursor.markDelete(ledger.getLastConfirmedEntry()); assertEventuallyTrue(() -> ledger.getLedgersInfoAsList().size() == 1); Assert.assertEquals(ledger.getLedgersInfoAsList().get(0).getLedgerId(), secondLedger); assertEventuallyTrue(() -> offloader.deletedOffloads().contains(firstLedger)); }
long firstLedgerId = ledger.getLedgersInfoAsList().get(0).getLedgerId(); ledger.offloadPrefix(ledger.getLastConfirmedEntry());
long firstLedgerId = ledger.getLedgersInfoAsList().get(0).getLedgerId(); ledger.offloadPrefix(ledger.getLastConfirmedEntry());
long firstLedgerId = ledger.getLedgersInfoAsList().get(0).getLedgerId(); ledger.offloadPrefix(ledger.getLastConfirmedEntry());
ledger.offloadPrefix(ledger.getLastConfirmedEntry()); } catch (ManagedLedgerException mle) {
ledger.offloadPrefix(p); Assert.fail("Shouldn't have succeeded"); } catch (ManagedLedgerException.OffloadInProgressException e) { ledger.offloadPrefix(p);
ledger.offloadPrefix(ledger.getLastConfirmedEntry());