/** * Creates a new instance of the BookKeeper log class. * * @param containerId The Id of the Container whose BookKeeperLog to open. * @param zkClient A reference to the CuratorFramework client to use. * @param bookKeeper A reference to the BookKeeper client to use. * @param config Configuration to use. * @param executorService An Executor to use for async operations. */ BookKeeperLog(int containerId, CuratorFramework zkClient, BookKeeper bookKeeper, BookKeeperConfig config, ScheduledExecutorService executorService) { Preconditions.checkArgument(containerId >= 0, "containerId must be a non-negative integer."); this.zkClient = Preconditions.checkNotNull(zkClient, "zkClient"); this.bookKeeper = Preconditions.checkNotNull(bookKeeper, "bookKeeper"); this.config = Preconditions.checkNotNull(config, "config"); this.executorService = Preconditions.checkNotNull(executorService, "executorService"); this.closed = new AtomicBoolean(); this.logNodePath = HierarchyUtils.getPath(containerId, this.config.getZkHierarchyDepth()); this.traceObjectId = String.format("Log[%d]", containerId); this.writes = new WriteQueue(); val retry = createRetryPolicy(this.config.getMaxWriteAttempts(), this.config.getBkWriteTimeoutMillis()); this.writeProcessor = new SequentialAsyncProcessor(this::processWritesSync, retry, this::handleWriteProcessorFailures, this.executorService); this.rolloverProcessor = new SequentialAsyncProcessor(this::rollover, retry, this::handleRolloverFailure, this.executorService); this.metrics = new BookKeeperMetrics.BookKeeperLog(containerId); this.metricReporter = this.executorService.scheduleWithFixedDelay(this::reportMetrics, REPORT_INTERVAL, REPORT_INTERVAL, TimeUnit.MILLISECONDS); }
/** * Write Processor main loop. This method is not thread safe and should only be invoked as part of the Write Processor. */ private void processWritesSync() { if (this.closed.get()) { // BookKeeperLog is closed. No point in trying anything else. return; } if (getWriteLedger().ledger.isClosed()) { // Current ledger is closed. Execute the rollover processor to safely create a new ledger. This will reinvoke // the write processor upon finish, so the writes can be reattempted. this.rolloverProcessor.runAsync(); } else if (!processPendingWrites() && !this.closed.get()) { // We were not able to complete execution of all writes. Try again. this.writeProcessor.runAsync(); } }
@Override public void close() { if (!this.closed.getAndSet(true)) { this.metricReporter.cancel(true); this.metrics.close(); this.rolloverProcessor.close(); this.writeProcessor.close(); // Close active ledger. WriteLedger writeLedger; synchronized (this.lock) { writeLedger = this.writeLedger; this.writeLedger = null; this.logMetadata = null; } // Close the write queue and cancel the pending writes. this.writes.close().forEach(w -> w.fail(new CancellationException("BookKeeperLog has been closed."), true)); if (writeLedger != null) { try { Ledgers.close(writeLedger.ledger); } catch (DurableDataLogException bkEx) { log.error("{}: Unable to close LedgerHandle for Ledger {}.", this.traceObjectId, writeLedger.ledger.getId(), bkEx); } } log.info("{}: Closed.", this.traceObjectId); } }
.retryWhen(t -> true); val error = new AtomicReference<Throwable>(); val p = new SequentialAsyncProcessor( () -> { count.incrementAndGet(); p.runAsync();
/** * Executes one instance of the task, or queues it up at most once should the task be currently running. */ void runAsync() { // Determine if a task is running. If so, record the fact we want to have it run again, otherwise reserve our spot. synchronized (this) { Exceptions.checkNotClosed(this.closed, this); if (this.running) { this.runAgain = true; return; } this.running = true; } // Execute the task. runInternal(); }
}); val error = new CompletableFuture<Throwable>(); val p = new SequentialAsyncProcessor( () -> { count.incrementAndGet(); p.runAsync();
private void runInternal() { this.retry.runInExecutor(this.runnable, this.executor) .whenComplete((r, ex) -> { if (ex != null) { // If we were unable to execute after all retries, invoke the failure callback. Callbacks.invokeSafely(this.failureCallback, ex, null); } boolean loopAgain; synchronized (this) { this.running = this.runAgain && !this.closed; this.runAgain = false; loopAgain = this.running; } if (loopAgain) { runInternal(); } }); }
this.rolloverProcessor.runAsync();
this.writeProcessor.runAsync(); } catch (ObjectClosedException ex) {
@Override public CompletableFuture<LogAddress> append(ArrayView data, Duration timeout) { ensurePreconditions(); long traceId = LoggerHelpers.traceEnterWithContext(log, this.traceObjectId, "append", data.getLength()); if (data.getLength() > getMaxAppendLength()) { return Futures.failedFuture(new WriteTooLongException(data.getLength(), getMaxAppendLength())); } Timer timer = new Timer(); // Queue up the write. CompletableFuture<LogAddress> result = new CompletableFuture<>(); this.writes.add(new Write(data, getWriteLedger(), result)); // Trigger Write Processor. this.writeProcessor.runAsync(); // Post append tasks. We do not need to wait for these to happen before returning the call. result.whenCompleteAsync((address, ex) -> { if (ex != null) { handleWriteException(ex); } else { // Update metrics and take care of other logging tasks. this.metrics.writeCompleted(timer.getElapsed()); LoggerHelpers.traceLeave(log, this.traceObjectId, "append", traceId, data.getLength(), address); } }, this.executorService); return result; }