if (log.isDebugEnabled()) { log.debug("[{}] [{}] Deleting individual messages at {}. Current status: {} - md-position: {}", ledger.getName(), name, positions, individualDeletedMessages, markDeletePosition); if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(position) < 0) { if (log.isDebugEnabled()) { log.debug( "[{}] Failed mark delete due to invalid markDelete {} is ahead of last-confirmed-entry {} for cursor [{}]", ledger.getName(), position, ledger.getLastConfirmedEntry(), name); log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position); PositionImpl previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.add(Range.openClosed(previousPosition, position)); ++messagesConsumedCounter; log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); .getNumberOfEntries(Range.openClosed(markDeletePosition, range.lowerEndpoint())) <= 0) { log.debug("[{}] Found a position range to mark delete for cursor {}: {} ", ledger.getName(), name, range); log.warn("[{}] [{}] Error while updating individualDeletedMessages [{}]", ledger.getName(), name, e.getMessage(), e); callback.deleteFailed(getManagedLedgerException(e), ctx); log.warn("[{}] [{}] Error doing asyncDelete [{}]", ledger.getName(), name, e.getMessage(), e);
@Override public long getEstimatedBacklogSize() { PositionImpl pos = getMarkDeletePositionOfSlowestConsumer(); size = getTotalSize(); size -= ledgers.values().stream().filter(li -> li.getLedgerId() < slowestConsumerLedgerId) .mapToLong(li -> li.getSize()).sum(); if (pos.compareTo(getMarkDeletePositionOfSlowestConsumer()) == 0) { pos = getMarkDeletePositionOfSlowestConsumer(); continue; size -= consumedLedgerSize(currentLedgerSize, currentLedgerEntries, numEntries); return size; } else { size -= consumedLedgerSize(ledgerInfo.getSize(), ledgerInfo.getEntries(), numEntries); return size;
@Override public long getNumberOfMessagesInBacklog() { long count = 0; for (ManagedCursor cursor : managedLedger.getCursors()) { count += cursor.getNumberOfEntriesInBacklog(); } return count; }
@Override public long getStoredMessagesSize() { return managedLedger.getTotalSize() * managedLedger.getConfig().getWriteQuorumSize(); }
private void recoverCursor(PositionImpl mdPosition) { Pair<PositionImpl, Long> lastEntryAndCounter = ledger.getLastPositionAndCounter(); this.readPosition = ledger.getNextValidPosition(mdPosition); markDeletePosition = mdPosition; // Initialize the counter such that the difference between the messages written on the ML and the // messagesConsumed is equal to the current backlog (negated). long initialBacklog = readPosition.compareTo(lastEntryAndCounter.getLeft()) < 0 ? ledger.getNumberOfEntries(Range.closed(readPosition, lastEntryAndCounter.getLeft())) : 0; messagesConsumedCounter = lastEntryAndCounter.getRight() - initialBacklog; }
/** * Get the first position written in the managed ledger, alongside with the associated counter */ Pair<PositionImpl, Long> getFirstPositionAndCounter() { PositionImpl pos; long count; Pair<PositionImpl, Long> lastPositionAndCounter; do { pos = getFirstPosition(); lastPositionAndCounter = getLastPositionAndCounter(); count = lastPositionAndCounter.getRight() - getNumberOfEntries(Range.openClosed(pos, lastPositionAndCounter.getLeft())); } while (pos.compareTo(getFirstPosition()) != 0 || lastPositionAndCounter.getLeft().compareTo(getLastPosition()) != 0); return Pair.of(pos, count); }
stats.entriesAddedCounter = ml.getEntriesAddedCounter(); stats.numberOfEntries = ml.getNumberOfEntries(); stats.totalSize = ml.getTotalSize(); stats.currentLedgerEntries = ml.getCurrentLedgerEntries(); stats.currentLedgerSize = ml.getCurrentLedgerSize(); stats.lastLedgerCreatedTimestamp = DATE_FORMAT.format(Instant.ofEpochMilli(ml.getLastLedgerCreatedTimestamp())); if (ml.getLastLedgerCreationFailureTimestamp() != 0) { stats.lastLedgerCreationFailureTimestamp = DATE_FORMAT .format(Instant.ofEpochMilli(ml.getLastLedgerCreationFailureTimestamp())); stats.waitingCursorsCount = ml.getWaitingCursorsCount(); stats.pendingAddEntriesCount = ml.getPendingAddEntriesCount(); stats.lastConfirmedEntry = ml.getLastConfirmedEntry().toString(); stats.state = ml.getState().toString(); ml.getLedgersInfo().forEach((id, li) -> { LedgerInfo info = new LedgerInfo(); info.ledgerId = li.getLedgerId(); ml.getCursors().forEach(c -> { ManagedCursorImpl cursor = (ManagedCursorImpl) c; CursorStats cs = new CursorStats();
@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 discardEmptyLedgersOnClose() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger"); ManagedCursor c1 = ledger.openCursor("c1"); ledger.addEntry("entry".getBytes()); assertEquals(ledger.getLedgersInfoAsList().size(), 1); c1.close(); ledger.close(); // re-open ledger = (ManagedLedgerImpl) factory.open("my_test_ledger"); assertEquals(ledger.getLedgersInfoAsList().size(), 2); // 1 ledger with 1 entry and the current writing ledger c1.close(); ledger.close(); // re-open, now the previous empty ledger should have been discarded ledger = (ManagedLedgerImpl) factory.open("my_test_ledger"); assertEquals(ledger.getLedgersInfoAsList().size(), 2); // 1 ledger with 1 entry, and the current // writing ledger }
@Test public void testConsumerSubscriptionInitializePosition() throws Exception{ final int MAX_ENTRY_PER_LEDGER = 2; ManagedLedgerConfig config = new ManagedLedgerConfig().setMaxEntriesPerLedger(MAX_ENTRY_PER_LEDGER); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("lastest_earliest_ledger", config); final int totalInsertedEntries = 20; for (int i = 0; i < totalInsertedEntries; i++) { String content = "entry" + i; // 5 bytes ledger.addEntry(content.getBytes()); } // Open Cursor also adds cursor into activeCursor-container ManagedCursor latestCursor = ledger.openCursor("c1", InitialPosition.Latest); ManagedCursor earliestCursor = ledger.openCursor("c2", InitialPosition.Earliest); // Since getReadPosition returns the next position, we decrease the entryId by 1 PositionImpl p1 = (PositionImpl) latestCursor.getReadPosition(); PositionImpl p2 = (PositionImpl) earliestCursor.getReadPosition(); Pair<PositionImpl, Long> latestPositionAndCounter = ledger.getLastPositionAndCounter(); Pair<PositionImpl, Long> earliestPositionAndCounter = ledger.getFirstPositionAndCounter(); assertEquals(latestPositionAndCounter.getLeft().getNext(), p1); assertEquals(earliestPositionAndCounter.getLeft().getNext(), p2); assertEquals(latestPositionAndCounter.getRight().longValue(), totalInsertedEntries); assertEquals(earliestPositionAndCounter.getRight().longValue(), totalInsertedEntries - earliestCursor.getNumberOfEntriesInBacklog()); ledger.close(); }
public long getNumberOfEntries(Range<PositionImpl> range) { return this.ledger.getNumberOfEntries(range); } }
@Test // (timeOut = 20000) public void asyncOpenClosedLedger() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my-closed-ledger"); ManagedCursor c1 = ledger.openCursor("c1"); ledger.addEntry("dummy-entry-1".getBytes(Encoding)); c1.close(); assertEquals(ledger.getNumberOfEntries(), 1); ledger.setFenced(); final CountDownLatch counter = new CountDownLatch(1); class Result { ManagedLedger instance1 = null; } final Result result = new Result(); factory.asyncOpen("my-closed-ledger", new OpenLedgerCallback() { @Override public void openLedgerComplete(ManagedLedger ledger, Object ctx) { result.instance1 = ledger; counter.countDown(); } @Override public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } }, null); counter.await(); assertNotNull(result.instance1); ManagedCursor c2 = result.instance1.openCursor("c1"); List<Entry> entries = c2.readEntries(1); assertEquals(entries.size(), 1); entries.forEach(e -> e.release()); }
try { ManagedLedgerImpl l = existingFuture.get(); if (l.getState().equals(State.Fenced.toString()) || l.getState().equals(State.Closed.toString())) { l.getState()); ledgers.remove(name, existingFuture); final ManagedLedgerImpl newledger = new ManagedLedgerImpl(this, bookKeeper, store, config, scheduledExecutor, orderedExecutor, name); newledger.initialize(new ManagedLedgerInitializeLedgerCallback() { @Override public void initializeComplete() {
@Test public void totalSizeTest() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); conf.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", conf); ManagedCursor c1 = ledger.openCursor("c1"); ledger.addEntry(new byte[10], 1, 8); assertEquals(ledger.getTotalSize(), 8); PositionImpl p2 = (PositionImpl) ledger.addEntry(new byte[12], 2, 5); assertEquals(ledger.getTotalSize(), 13); c1.markDelete(new PositionImpl(p2.getLedgerId(), -1)); // Wait for background trimming Thread.sleep(400); assertEquals(ledger.getTotalSize(), 5); }
@Test public void discardEmptyLedgersOnError() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger"); assertEquals(ledger.getLedgersInfoAsList().size(), 1); bkc.failNow(BKException.Code.NoBookieAvailableException); zkc.failNow(Code.CONNECTIONLOSS); try { ledger.addEntry("entry".getBytes()); fail("Should have received exception"); } catch (ManagedLedgerException e) { // Ok } assertEquals(ledger.getLedgersInfoAsList().size(), 0); // Next write should fail as well try { ledger.addEntry("entry".getBytes()); fail("Should have received exception"); } catch (ManagedLedgerException e) { // Ok } assertEquals(ledger.getLedgersInfoAsList().size(), 0); assertEquals(ledger.getNumberOfEntries(), 0); }
@Test public void recoverLongTimeAfterMultipleWriteErrors() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("recoverLongTimeAfterMultipleWriteErrors"); ManagedCursor cursor = ledger.openCursor("c1"); ledger.asyncAddEntry("entry-1".getBytes(), cb, null); ledger.asyncAddEntry("entry-2".getBytes(), cb, null); assertEquals(ledger.getLedgersInfoAsList().size(), 1); ledger.addEntry("entry-3".getBytes());
public long getNumberOfEntriesInStorage() { return ledger.getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition().getNext())); }
protected long getNumberOfEntries(Range<PositionImpl> range) { long allEntries = ledger.getNumberOfEntries(range); if (log.isDebugEnabled()) { log.debug("[{}] getNumberOfEntries. {} allEntries: {}", ledger.getName(), range, allEntries); } long deletedEntries = 0; lock.readLock().lock(); try { for (Range<PositionImpl> r : individualDeletedMessages.asRanges()) { if (r.isConnected(range)) { Range<PositionImpl> commonEntries = r.intersection(range); long commonCount = ledger.getNumberOfEntries(commonEntries); if (log.isDebugEnabled()) { log.debug("[{}] [{}] Discounting {} entries for already deleted range {}", ledger.getName(), name, commonCount, commonEntries); } deletedEntries += commonCount; } } } finally { lock.readLock().unlock(); } if (log.isDebugEnabled()) { log.debug("[{}] Found {} entries - deleted: {}", ledger.getName(), allEntries - deletedEntries, deletedEntries); } return allEntries - deletedEntries; }
@Test public void testGetNextValidPosition() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); conf.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testGetNextValidPosition", conf); ManagedCursor c1 = ledger.openCursor("c1"); PositionImpl p1 = (PositionImpl) ledger.addEntry("entry1".getBytes()); PositionImpl p2 = (PositionImpl) ledger.addEntry("entry2".getBytes()); PositionImpl p3 = (PositionImpl) ledger.addEntry("entry3".getBytes()); assertEquals(ledger.getNextValidPosition((PositionImpl) c1.getMarkDeletedPosition()), p1); assertEquals(ledger.getNextValidPosition(p1), p2); assertEquals(ledger.getNextValidPosition(p3), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); }
@Test public void testEstimatedBacklogSize() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testEstimatedBacklogSize"); ManagedCursor c1 = ledger.openCursor("c1"); ledger.addEntry(new byte[1024]); Position position2 = ledger.addEntry(new byte[1024]); ledger.addEntry(new byte[1024]); ledger.addEntry(new byte[1024]); Position lastPosition = ledger.addEntry(new byte[1024]); long backlog = ledger.getEstimatedBacklogSize(); assertEquals(backlog, 1024 * 5); List<Entry> entries = c1.readEntries(2); entries.forEach(Entry::release); c1.markDelete(position2); backlog = ledger.getEstimatedBacklogSize(); assertEquals(backlog, 1024 * 3); entries = c1.readEntries(3); entries.forEach(Entry::release); c1.markDelete(lastPosition); backlog = ledger.getEstimatedBacklogSize(); assertEquals(backlog, 0); }