/** * 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); }
@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); } }
@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; }
private void reportMetrics() { this.metrics.ledgerCount(getLogMetadata().getLedgers().size()); this.metrics.queueStats(this.writes.getStatistics()); }
/** * Completes the given Write and makes any necessary internal updates. * * @param write The write to complete. */ private void completeWrite(Write write) { Timer t = write.complete(); if (t != null) { this.metrics.bookKeeperWriteCompleted(write.data.getLength(), t.getElapsed()); } }