void notifyCursors() { while (true) { final ManagedCursorImpl waitingCursor = waitingCursors.poll(); if (waitingCursor == null) { break; } executor.execute(safeRun(() -> waitingCursor.notifyEntriesAvailable())); } }
@Override public void asyncAddEntry(ByteBuf buffer, AddEntryCallback callback, Object ctx) { if (log.isDebugEnabled()) { log.debug("[{}] asyncAddEntry size={} state={}", name, buffer.readableBytes(), state); } OpAddEntry addOperation = OpAddEntry.create(this, buffer, callback, ctx); // Jump to specific thread to avoid contention from writers writing from different threads executor.executeOrdered(name, safeRun(() -> { pendingAddEntries.add(addOperation); internalAsyncAddEntry(addOperation); })); }
private void trimConsumedLedgersInBackground(CompletableFuture<?> promise) { executor.executeOrdered(name, safeRun(() -> { internalTrimConsumedLedgers(promise); })); }
private void scheduleDeferredTrimming(CompletableFuture<?> promise) { scheduledExecutor.schedule(safeRun(() -> trimConsumedLedgersInBackground(promise)), 100, TimeUnit.MILLISECONDS); }
boolean hasSpaceInCache() { long currentSize = this.currentSize.get(); // Trigger a single eviction in background. While the eviction is running we stop inserting entries in the cache if (currentSize > evictionTriggerThreshold && evictionInProgress.compareAndSet(false, true)) { mlFactory.scheduledExecutor.execute(safeRun(() -> { // Trigger a new cache eviction cycle to bring the used memory below the cacheEvictionWatermark // percentage limit long sizeToEvict = currentSize - (long) (maxSize * cacheEvictionWatermak); long startTime = System.nanoTime(); log.info("Triggering cache eviction. total size: {} Mb -- Need to discard: {} Mb", currentSize / MB, sizeToEvict / MB); try { evictionPolicy.doEviction(Lists.newArrayList(caches.values()), sizeToEvict); long endTime = System.nanoTime(); double durationMs = TimeUnit.NANOSECONDS.toMicros(endTime - startTime) / 1000.0; log.info("Eviction completed. Removed {} Mb in {} ms", (currentSize - this.currentSize.get()) / MB, durationMs); } finally { mlFactoryMBean.recordCacheEviction(); evictionInProgress.set(false); } })); } return currentSize < maxSize; }
private void asyncDeleteLedger(long ledgerId, long retry) { if (retry <= 0) { log.warn("[{}] Failed to delete ledger after retries {}", name, ledgerId); return; } bookKeeper.asyncDeleteLedger(ledgerId, (rc, ctx) -> { if (rc == BKException.Code.NoSuchLedgerExistsException) { log.warn("[{}] Ledger was already deleted {}", name, ledgerId); } else if (rc != BKException.Code.OK) { log.error("[{}] Error deleting ledger {}", name, ledgerId, BKException.getMessage(rc)); scheduledExecutor.schedule(safeRun(() -> { asyncDeleteLedger(ledgerId, retry - 1); }), DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC, TimeUnit.SECONDS); } else { if (log.isDebugEnabled()) { log.debug("[{}] Deleted ledger {}", name, ledgerId); } } }, null); }
private void maybeOffloadInBackground(CompletableFuture<PositionImpl> promise) { if (config.getOffloadAutoTriggerSizeThresholdBytes() >= 0) { executor.executeOrdered(name, safeRun(() -> maybeOffload(promise))); } }
@Override public void getCursors(final String ledgerName, final MetaStoreCallback<List<String>> callback) { if (log.isDebugEnabled()) { log.debug("[{}] Get cursors list", ledgerName); } zk.getChildren(prefix + ledgerName, false, (rc, path, ctx, children, stat) -> executor.executeOrdered(ledgerName, safeRun(() -> { if (log.isDebugEnabled()) { log.debug("[{}] getConsumers complete rc={} children={}", ledgerName, Code.get(rc), children); } if (rc != Code.OK.intValue()) { callback.operationFailed(new MetaStoreException(KeeperException.create(Code.get(rc)))); return; } if (log.isDebugEnabled()) { log.debug("[{}] Get childrend completed version={}", ledgerName, stat.getVersion()); } callback.operationComplete(children, new ZKStat(stat)); })), null); }
safeRun(() -> tryTransformLedgerInfo(ledgerId, transformation, finalPromise)), 100, TimeUnit.MILLISECONDS); } else { // lock acquired
@Override public void removeManagedLedger(String ledgerName, MetaStoreCallback<Void> callback) { log.info("[{}] Remove ManagedLedger", ledgerName); zk.delete(prefix + ledgerName, -1, (rc, path, ctx) -> executor.executeOrdered(ledgerName, safeRun(() -> { if (log.isDebugEnabled()) { log.debug("[{}] zk delete done. rc={}", ledgerName, Code.get(rc)); } if (rc == Code.OK.intValue()) { callback.operationComplete(null, null); } else { callback.operationFailed(new MetaStoreException(KeeperException.create(Code.get(rc)))); } })), null); }
private void maybeOffload(CompletableFuture<PositionImpl> finalPromise) { if (!offloadMutex.tryLock()) { scheduledExecutor.schedule(safeRun(() -> maybeOffloadInBackground(finalPromise)), 100, TimeUnit.MILLISECONDS); } else {
@Override public void asyncRemoveCursor(final String ledgerName, final String consumerName, final MetaStoreCallback<Void> callback) { log.info("[{}] Remove consumer={}", ledgerName, consumerName); zk.delete(prefix + ledgerName + "/" + consumerName, -1, (rc, path, ctx) -> executor.executeOrdered(ledgerName, safeRun(() -> { if (log.isDebugEnabled()) { log.debug("[{}] [{}] zk delete done. rc={}", ledgerName, consumerName, Code.get(rc)); } if (rc == Code.OK.intValue()) { callback.operationComplete(null, null); } else { callback.operationFailed(new MetaStoreException(KeeperException.create(Code.get(rc)))); } })), null); }
@Override public void asyncGetCursorInfo(String ledgerName, String consumerName, final MetaStoreCallback<ManagedCursorInfo> callback) { String path = prefix + ledgerName + "/" + consumerName; if (log.isDebugEnabled()) { log.debug("Reading from {}", path); } zk.getData(path, false, (rc, path1, ctx, data, stat) -> executor.executeOrdered(ledgerName, safeRun(() -> { if (rc != Code.OK.intValue()) { callback.operationFailed(new MetaStoreException(KeeperException.create(Code.get(rc)))); } else { try { ManagedCursorInfo info = parseManagedCursorInfo(data); callback.operationComplete(info, new ZKStat(stat)); } catch (ParseException | InvalidProtocolBufferException e) { callback.operationFailed(new MetaStoreException(e)); } } })), null); if (log.isDebugEnabled()) { log.debug("Reading from {} ok", path); } }
/** * It handles add failure on the given ledger. it can be triggered when add-entry fails or times out. * * @param ledger */ void handleAddFailure(final LedgerHandle ledger) { // If we get a write error, we will try to create a new ledger and re-submit the pending writes. If the // ledger creation fails (persistent bk failure, another instanche owning the ML, ...), then the writes will // be marked as failed. ml.mbean.recordAddEntryError(); ml.getExecutor().executeOrdered(ml.getName(), SafeRun.safeRun(() -> { // Force the creation of a new ledger. Doing it in a background thread to avoid acquiring ML lock // from a BK callback. ml.ledgerClosed(ledger); })); }
ledger.getScheduledExecutor().schedule(safeRun(() -> { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Re-trying the read at position {}", ledger.getName(), name, op.readPosition);
void checkReadCompletion() { if (entries.size() < count && cursor.hasMoreEntries()) { // We still have more entries to read from the next ledger, schedule a new async operation if (nextReadPosition.getLedgerId() != readPosition.getLedgerId()) { cursor.ledger.startReadOperationOnLedger(nextReadPosition); } // Schedule next read in a different thread cursor.ledger.getExecutor().execute(safeRun(() -> { readPosition = cursor.ledger.startReadOperationOnLedger(nextReadPosition); cursor.ledger.asyncReadEntries(OpReadEntry.this); })); } else { // The reading was already completed, release resources and trigger callback try { cursor.readOperationCompleted(); } finally { cursor.ledger.getExecutor().executeOrdered(cursor.ledger.getName(), safeRun(() -> { callback.readEntriesComplete(entries, ctx); recycle(); })); } } }
@Override public void asyncResetCursor(Position newPos, AsyncCallbacks.ResetCursorCallback callback) { checkArgument(newPos instanceof PositionImpl); final PositionImpl newPosition = (PositionImpl) newPos; // order trim and reset operations on a ledger ledger.getExecutor().executeOrdered(ledger.getName(), safeRun(() -> { if (ledger.isValidPosition(newPosition) || newPosition.equals(PositionImpl.earliest) || newPosition.equals(PositionImpl.latest)) { internalResetCursor(newPosition, callback); } else { // caller (replay) should handle this error and retry cursor reset callback.resetFailed(new ManagedLedgerException.InvalidCursorPositionException(newPosition.toString()), newPosition); } })); }
(rc, path1, ctx, name) -> executor.executeOrdered(ledgerName, safeRun(() -> { if (rc != Code.OK.intValue()) { log.warn("[{}] Error creating cosumer {} node on meta-data store with {}: ", ledgerName, (rc, path1, ctx, stat1) -> executor.executeOrdered(ledgerName, safeRun(() -> { if (rc == Code.BADVERSION.intValue()) { callback.operationFailed(new BadVersionException(KeeperException.create(Code.get(rc))));
final long id = ledgers.lastKey(); OpenCallback opencb = (rc, lh, ctx1) -> { executor.executeOrdered(name, safeRun(() -> { mbean.endDataLedgerOpenOp(); if (log.isDebugEnabled()) {
executor.submit(safeRun(() -> { try { cursor.asyncDelete(addedEntries.get(iteration.getAndIncrement()), new DeleteCallback() {