try { int attemptCount = w.beginAttempt(); if (attemptCount > this.config.getMaxWriteAttempts()) { throw new RetriesExhaustedException(w.getFailureCause()); w.getWriteLedger().ledger.asyncAddEntry(w.data.array(), w.data.arrayOffset(), w.data.getLength(), this::addCallback, w); } catch (Throwable ex) { w.fail(ex, isFinal); toExecute.get(j).fail(new DurableDataLogException("Previous write failed.", ex), isFinal);
/** * 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()); } }
/** * Removes all the completed writes (whether successful or failed) from the beginning of the queue, until the first * non-completed item is encountered or the queue is empty. * * @return A CleanupStatus representing the state of the Operation. If there were failed writes, this will be WriteFailed, * otherwise it will be one of QueueEmpty or QueueNotEmpty, depending on the final state of the queue when this method * finishes. */ synchronized CleanupStatus removeFinishedWrites() { Exceptions.checkNotClosed(this.closed, this); long currentTime = this.timeSupplier.get(); long totalElapsed = 0; int removedCount = 0; boolean failedWrite = false; while (!this.writes.isEmpty() && this.writes.peekFirst().isDone()) { Write w = this.writes.removeFirst(); this.totalLength = Math.max(0, this.totalLength - w.data.getLength()); removedCount++; totalElapsed += currentTime - w.getQueueAddedTimestamp(); failedWrite |= w.getFailureCause() != null; } if (removedCount > 0) { this.lastDurationMillis = (int) (totalElapsed / removedCount / AbstractTimer.NANOS_TO_MILLIS); } return failedWrite ? CleanupStatus.WriteFailed : this.writes.isEmpty() ? CleanupStatus.QueueEmpty : CleanupStatus.QueueNotEmpty; }
@Override public String toString() { return String.format("LedgerId = %s, Length = %s, Attempts = %s, InProgress = %s, Done = %s, Failed %s", this.writeLedger.get().metadata.getLedgerId(), this.data.getLength(), this.attemptCount, isInProgress(), isDone(), this.failureCause.get() != null); }
if (writes.size() == 0 || !writes.get(0).getWriteLedger().ledger.isClosed()) { boolean anythingChanged = false; for (Write w : writes) { if (w.isDone() || !w.getWriteLedger().ledger.isClosed()) { continue; long lac = fetchLastAddConfirmed(w.getWriteLedger(), lastAddsConfirmed); if (w.getEntryId() >= 0 && w.getEntryId() <= lac) { } else if (currentLedger.ledger.getId() != w.getWriteLedger().ledger.getId()) { w.setWriteLedger(currentLedger); anythingChanged = true;
for (int i = 0; i < ITEM_COUNT; i++) { time.addAndGet(timeIncrement); val w = new Write(new ByteArraySegment(new byte[i]), new TestWriteLedger(i), new CompletableFuture<>()); if (i % 2 == 0) { w.setEntryId(i); w.complete(); if (!write.isDone()) { val result1 = q.removeFinishedWrites(); Assert.assertEquals("Unexpected value from removeFinishedWrites when there were writes left in the queue.", write.setEntryId(time.get()); write.complete(); long expectedElapsed = write.getQueueAddedTimestamp(); int removed = 1; while (!writes.isEmpty() && writes.peekFirst().isDone()) { expectedElapsed += writes.pollFirst().getQueueAddedTimestamp(); removed++; val w3 = new Write(new ByteArraySegment(new byte[1]), new TestWriteLedger(0), new CompletableFuture<>()); q.add(w3); w3.fail(new IntentionalException(), true); val result3 = q.removeFinishedWrites(); Assert.assertEquals("Unexpected value from removeFinishedWrites when there were failed writes.",
val w = new Write(new ByteArraySegment(new byte[i]), new TestWriteLedger(ledgerId), new CompletableFuture<>()); q.add(w); writes.add(w); writes.get(0).setEntryId(0); writes.get(0).complete(); writes.get(1).beginAttempt(); val result1 = q.getWritesToExecute(Long.MAX_VALUE); writes.get(3).beginAttempt(); val result2 = q.getWritesToExecute(Long.MAX_VALUE); Assert.assertEquals("Unexpected writes fetched when in-progress writes exist after non-in-progress writes.", writes.get(i).setEntryId(i); writes.get(i).complete(); writes.subList(beginIndex, ledgerChangeIndex), result3, Object::equals); result3.forEach(w -> w.setEntryId(0)); result3.forEach(Write::complete); q.removeFinishedWrites();
Write write = (Write) ctx; try { assert handle.getId() == write.getWriteLedger().ledger.getId() : "Handle.Id mismatch: " + write.getWriteLedger().ledger.getId() + " vs " + handle.getId(); write.setEntryId(entryId); if (rc == 0) { } catch (Throwable ex) { write.fail(ex, !isRetryable(ex)); } finally {
long firstLedgerId = this.writes.peekFirst().getWriteLedger().metadata.getLedgerId(); boolean canSkip = true; if (write.isInProgress()) { if (!canSkip) { } else if (write.getWriteLedger().metadata.getLedgerId() != firstLedgerId) { } else if (!write.isDone()) { canSkip = false; result.add(write);
@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; }
@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); } }
/** * Gets a snapshot of the queue internals. * * @return The snapshot, including Queue Size, Item Fill Rate and elapsed time of the oldest item. */ synchronized QueueStats getStatistics() { int size = this.writes.size(); double fillRatio = calculateFillRatio(this.totalLength, size); int processingTime = this.lastDurationMillis; if (processingTime == 0 && size > 0) { // We get in here when this method is invoked prior to any operation being completed. Since lastDurationMillis // is only set when an item is completed, in this special case we just estimate based on the amount of time // the first item in the queue has been added. processingTime = (int) ((this.timeSupplier.get() - this.writes.peekFirst().getQueueAddedTimestamp()) / AbstractTimer.NANOS_TO_MILLIS); } return new QueueStats(size, fillRatio, processingTime); }
/** * Indicates that this write completed successfully. This will set the final result on the externalCompletion future. */ Timer complete() { Preconditions.checkState(this.entryId.get() >= 0, "entryId not set; cannot complete Write."); this.failureCause.set(null); this.result.complete(new LedgerAddress(this.writeLedger.get().metadata, this.entryId.get())); return endAttempt(); }
val expectedWrites = new ArrayList<Write>(); for (int i = 0; i < ITEM_COUNT; i++) { val w = new Write(new ByteArraySegment(new byte[i]), new TestWriteLedger(i), CompletableFuture.completedFuture(null)); q.add(w); expectedWrites.add(w); () -> q.add(new Write(new ByteArraySegment(new byte[1]), new TestWriteLedger(0), CompletableFuture.completedFuture(null))), ex -> ex instanceof ObjectClosedException); AssertExtensions.assertThrows(
write.fail(ex, !isRetryable(ex));
/** * Indicates that this write failed. * * @param cause The failure cause. If null, the previous failure cause is preserved. * @param complete If true, the externalCompletion will be immediately be completed with the current failure cause. * If false, no completion will be done. */ void fail(Throwable cause, boolean complete) { if (cause != null) { Throwable e = this.failureCause.get(); if (e != null && e != cause) { cause.addSuppressed(e); } this.failureCause.set(cause); } endAttempt(); WriteLedger ledger = this.writeLedger.get(); if (ledger != null && ledger.isRolledOver()) { // Rollovers aren't really failures (they're caused by us). In that case, do not count this failure as an attempt. this.attemptCount.updateAndGet(v -> Math.max(0, v - 1)); } if (complete) { this.result.completeExceptionally(this.failureCause.get()); } }
/** * Tests the basic functionality of the add() method. */ @Test public void testAdd() { final int timeIncrement = 1234 * 1000; // Just over 1ms. AtomicLong time = new AtomicLong(); val q = new WriteQueue(time::get); val initialStats = q.getStatistics(); Assert.assertEquals("Unexpected getSize on empty queue.", 0, initialStats.getSize()); Assert.assertEquals("Unexpected getAverageFillRate on empty queue.", 0, initialStats.getAverageItemFillRatio(), 0); Assert.assertEquals("Unexpected getExpectedProcessingTimeMillis on empty queue.", 0, initialStats.getExpectedProcessingTimeMillis()); int expectedSize = 0; long firstItemTime = 0; for (int i = 0; i < ITEM_COUNT; i++) { time.addAndGet(timeIncrement); if (i == 0) { firstItemTime = time.get(); } int writeSize = i * 10000; q.add(new Write(new ByteArraySegment(new byte[writeSize]), new TestWriteLedger(i), CompletableFuture.completedFuture(null))); expectedSize += writeSize; val stats = q.getStatistics(); val expectedFillRatio = (double) expectedSize / stats.getSize() / BookKeeperConfig.MAX_APPEND_LENGTH; val expectedProcTime = (time.get() - firstItemTime) / AbstractTimer.NANOS_TO_MILLIS; Assert.assertEquals("Unexpected getSize.", i + 1, stats.getSize()); Assert.assertEquals("Unexpected getAverageFillRate.", expectedFillRatio, stats.getAverageItemFillRatio(), 0.01); Assert.assertEquals("Unexpected getExpectedProcessingTimeMillis.", expectedProcTime, stats.getExpectedProcessingTimeMillis()); } }