public long getCursorLedger() { LedgerHandle lh = cursorLedger; return lh != null ? lh.getId() : -1; }
@Override public long getLogSegmentId() { return lh.getId(); }
@Override public void run() { try { done.await(); } catch (InterruptedException e) { logger.info("Interrupted on stream reading ledger {} : ", lh.getId(), e); } } }
/** * It cancels timeout task and checks if add-entry operation is not completed yet. * * @return true if task is not already completed else returns false. */ private boolean checkAndCompleteTimeoutTask() { if (!COMPLETED_UPDATER.compareAndSet(this, FALSE, TRUE)) { if (log.isDebugEnabled()) { log.debug("Add-entry already completed for {}-{}", this.ledger != null ? this.ledger.getId() : -1, this.entryId); } return false; } return true; }
@Override public void operationComplete(Void result, Stat stat) { log.info("[{}] Updated cursor {} with ledger id {} md-position={} rd-position={}", ledger.getName(), name, lh.getId(), markDeletePosition, readPosition); final LedgerHandle oldLedger = cursorLedger; cursorLedger = lh; cursorLedgerStat = stat; // At this point the position had already been safely markdeleted callback.operationComplete(); asyncDeleteLedger(oldLedger); }
void invalidateLedgerHandle(ReadHandle ledgerHandle, Throwable t) { long ledgerId = ledgerHandle.getId(); if (currentLedger != null && ledgerId != currentLedger.getId()) { // remove handle from ledger cache since we got a (read) error ledgerCache.remove(ledgerId); if (log.isDebugEnabled()) { log.debug("[{}] Removed ledger {} from cache (after read error)", name, ledgerId, t); } } else { if (log.isDebugEnabled()) { log.debug("[{}] Ledger that encountered read error is current ledger", name, t); } } }
@Override public void closeComplete(int rc, LedgerHandle lh, Object ctx) { checkArgument(ledger.getId() == lh.getId(), "ledgerId %s doesn't match with acked ledgerId %s", ledger.getId(), lh.getId()); if (rc == BKException.Code.OK) { log.debug("Successfuly closed ledger {}", lh.getId()); } else { log.warn("Error when closing ledger {}. Status={}", lh.getId(), BKException.getMessage(rc)); } ml.ledgerClosed(lh); updateLatency(); AddEntryCallback cb = callbackUpdater.getAndSet(this, null); if (cb != null) { cb.addComplete(PositionImpl.get(lh.getId(), entryId), ctx); ml.notifyCursors(); this.recycle(); } }
@Override public void operationComplete() { log.info("[{}][{}] Updated md-position={} into cursor-ledger {}", ledger.getName(), name, markDeletePosition, cursorLedger.getId()); callback.closeComplete(ctx); }
@Override public void operationFailed(ManagedLedgerException e) { log.warn("[{}][{}] Failed to persist mark-delete position into cursor-ledger{}: {}", ledger.getName(), name, cursorLedger.getId(), e.getMessage()); callback.closeFailed(e, ctx); } });
void asyncReadEntry(PositionImpl position, ReadEntryCallback callback, Object ctx) { LedgerHandle currentLedger = this.currentLedger; if (log.isDebugEnabled()) { log.debug("[{}] Reading entry ledger {}: {}", name, position.getLedgerId(), position.getEntryId()); } if (position.getLedgerId() == currentLedger.getId()) { LedgerHandle ledger = currentLedger; asyncReadEntry(ledger, position, callback, ctx); } else { getLedgerHandle(position.getLedgerId()).thenAccept(ledger -> { asyncReadEntry(ledger, position, callback, ctx); }).exceptionally(ex -> { log.error("[{}] Error opening ledger for reading at position {} - {}", name, position, ex.getMessage()); callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(ex.getCause()), ctx); return null; }); } }
void asyncCloseCursorLedger(final AsyncCallbacks.CloseCallback callback, final Object ctx) { LedgerHandle lh = cursorLedger; ledger.mbean.startCursorLedgerCloseOp(); log.info("[{}] [{}] Closing metadata ledger {}", ledger.getName(), name, lh.getId()); lh.asyncClose(new CloseCallback() { @Override public void closeComplete(int rc, LedgerHandle lh, Object ctx) { ledger.mbean.endCursorLedgerCloseOp(); if (rc == BKException.Code.OK) { callback.closeComplete(ctx); } else { callback.closeFailed(createManagedLedgerException(rc), ctx); } } }, ctx); }
private void scheduleTimeoutTask() { long timeoutSec = config.getAddEntryTimeoutSeconds(); // disable timeout task checker if timeout <= 0 if (timeoutSec > 0) { this.timeoutTask = this.scheduledExecutor.scheduleAtFixedRate(() -> { OpAddEntry opAddEntry = pendingAddEntries.peek(); if (opAddEntry != null) { boolean isTimedOut = opAddEntry.lastInitTime != -1 && TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - opAddEntry.lastInitTime) >= timeoutSec && opAddEntry.completed == FALSE; if (isTimedOut) { log.error("Failed to add entry for ledger {} in time-out {} sec", (opAddEntry.ledger != null ? opAddEntry.ledger.getId() : -1), timeoutSec); opAddEntry.handleAddFailure(opAddEntry.ledger); } } }, config.getAddEntryTimeoutSeconds(), config.getAddEntryTimeoutSeconds(), TimeUnit.SECONDS); } }
long estimateBacklogFromPosition(PositionImpl pos) { synchronized (this) { LedgerInfo ledgerInfo = ledgers.get(pos.getLedgerId()); if (ledgerInfo == null) { return getTotalSize(); // position no longer in managed ledger, so return total size } long sizeBeforePosLedger = ledgers.values().stream().filter(li -> li.getLedgerId() < pos.getLedgerId()) .mapToLong(li -> li.getSize()).sum(); long size = getTotalSize() - sizeBeforePosLedger; if (pos.getLedgerId() == currentLedger.getId()) { return size - consumedLedgerSize(currentLedgerSize, currentLedgerEntries, pos.getEntryId()); } else { return size - consumedLedgerSize(ledgerInfo.getSize(), ledgerInfo.getEntries(), pos.getEntryId()); } } }
private LedgerHandle openLedger(LedgerHandle lh) throws Exception { return bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8)); }
private void fenceLedger(LedgerHandle lh) throws Exception { bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8)); }
private LedgerHandle openLedgerNoRecovery(LedgerHandle lh) throws Exception { return bkc.get().openLedgerNoRecovery(lh.getId(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8)); }
@Override public void run() { try { for (int i = 0; i < numLedgers; i++) { pool.allocate(); ZKTransaction txn = newTxn(); LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER)); FutureUtils.result(txn.execute()); lh.close(); allocatedLedgers.putIfAbsent(lh.getId(), lh); logger.info("[thread {}] allocate {}th ledger {}", new Object[] { tid, i, lh.getId() }); } } catch (Exception ioe) { numFailures.incrementAndGet(); } } };
@Test public void simpleTest() throws Exception { LedgerHandle ledger = bkc.createLedger(DigestType.MAC, SECRET.getBytes()); long ledgerId = ledger.getId(); log.info("Writing to ledger: {}", ledgerId); for (int i = 0; i < 10; i++) { String content = "entry-" + i; ledger.addEntry(content.getBytes(Encoding)); } ledger.close(); ledger = bkc.openLedger(ledgerId, DigestType.MAC, SECRET.getBytes()); Enumeration<LedgerEntry> entries = ledger.readEntries(0, 9); while (entries.hasMoreElements()) { LedgerEntry entry = entries.nextElement(); String content = new String(entry.getEntry(), Encoding); log.info("Entry {} lenght={} content='{}'", entry.getEntryId(), entry.getLength(), content); } ledger.close(); }
@Test(timeout = 60000) public void testCloseAllocatorDuringObtaining() throws Exception { String allocationPath = "/allocation2"; SimpleLedgerAllocator allocator = createAllocator(allocationPath); allocator.allocate(); ZKTransaction txn = newTxn(); // close during obtaining ledger. LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER)); Utils.close(allocator); byte[] data = zkc.get().getData(allocationPath, false, null); assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8))); // the ledger is not deleted bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes(UTF_8)); }