From 82cbcafa6513e8967dae7dfc535f0b8bf3534846 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 12:26:58 +0800 Subject: [PATCH 01/16] Introduce ReadEntriesContext to ensure callback is only called once --- .../mledger/ReadEntriesContext.java | 90 +++++++++++++++++++ .../impl/cache/PendingReadsManager.java | 58 ++++-------- .../impl/cache/RangeEntryCacheImpl.java | 88 ++++++++---------- .../impl/cache/PendingReadsManagerTest.java | 8 +- 4 files changed, 148 insertions(+), 96 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java new file mode 100644 index 0000000000000..d0621b422388a --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +/** + * It wraps the read entries' callback and context to ensure that the callback can only be triggered once. + */ +public class ReadEntriesContext { + + private AsyncCallbacks.ReadEntriesCallback callback; + private final Object ctx; + private Consumer> beforeCompleteCallback = null; + + public ReadEntriesContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { + this.callback = callback; + this.ctx = ctx; + } + + public void complete(List entries) { + AsyncCallbacks.ReadEntriesCallback callback; + Consumer> beforeCompleteCallback; + synchronized (this) { + beforeCompleteCallback = this.beforeCompleteCallback; + this.beforeCompleteCallback = null; + callback = this.callback; + this.callback = null; + } + if (beforeCompleteCallback != null) { + beforeCompleteCallback.accept(entries); + } + if (callback != null) { + callback.readEntriesComplete(entries, ctx); + } + } + + public void fail(Throwable throwable) { + AsyncCallbacks.ReadEntriesCallback callback; + Consumer> beforeCompleteCallback; + synchronized (this) { + beforeCompleteCallback = this.beforeCompleteCallback; + this.beforeCompleteCallback = null; + callback = this.callback; + this.callback = null; + } + if (beforeCompleteCallback != null) { + beforeCompleteCallback.accept(List.of()); + } + if (callback != null) { + callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } + } + + public synchronized void setBeforeCompleteCallback(Consumer> beforeCompleteCallback) { + this.beforeCompleteCallback = beforeCompleteCallback; + } + + public static ReadEntriesContext fromFuture(CompletableFuture> future) { + return new ReadEntriesContext(new AsyncCallbacks.ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + future.complete(entries); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + } +} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index 5f904f3cf8420..a956cf61fb913 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -18,7 +18,6 @@ */ package org.apache.bookkeeper.mledger.impl.cache; -import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; import io.prometheus.client.Counter; import java.util.ArrayList; import java.util.Collections; @@ -31,9 +30,8 @@ import java.util.function.IntSupplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.ReadEntriesContext; import org.apache.bookkeeper.mledger.impl.EntryImpl; /** @@ -132,7 +130,7 @@ PendingReadKey reminderOnRight(PendingReadKey other) { } - private record ReadEntriesCallbackWithContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx, + private record ReadEntriesCallbackWithContext(ReadEntriesContext readEntriesContext, long startEntry, long endEntry) { } @@ -248,12 +246,11 @@ public synchronized void attach(CompletableFuture> handle) { }); } - synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback, - Object ctx, long startEntry, long endEntry) { + synchronized boolean addListener(ReadEntriesContext context, long startEntry, long endEntry) { if (state == PendingReadState.COMPLETED) { return false; } - listeners.add(new ReadEntriesCallbackWithContext(callback, ctx, startEntry, endEntry)); + listeners.add(new ReadEntriesCallbackWithContext(context, startEntry, endEntry)); return true; } @@ -276,15 +273,14 @@ private void readEntriesComplete(List callbacks, if (first.startEntry == key.startEntry && first.endEntry == key.endEntry) { // perfect match, no copy, this is the most common case - first.callback.readEntriesComplete(entriesToReturn, first.ctx); + first.readEntriesContext.complete(entriesToReturn); } else { - first.callback.readEntriesComplete( - keepEntries(entriesToReturn, first.startEntry, first.endEntry), first.ctx); + first.readEntriesContext.complete(keepEntries(entriesToReturn, first.startEntry, first.endEntry)); } } else { for (ReadEntriesCallbackWithContext callback : callbacks) { - callback.callback.readEntriesComplete( - copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), callback.ctx); + callback.readEntriesContext.complete( + copyEntries(entriesToReturn, callback.startEntry, callback.endEntry)); } for (Entry entry : entriesToReturn) { // don't decrease the read count when these entries are released @@ -297,8 +293,7 @@ private void readEntriesComplete(List callbacks, // this method isn't synchronized since that could lead to deadlocks private void readEntriesFailed(List callbacks, Throwable error) { for (ReadEntriesCallbackWithContext callback : callbacks) { - ManagedLedgerException mlException = createManagedLedgerException(error); - callback.callback.readEntriesFailed(mlException, callback.ctx); + callback.readEntriesContext.fail(error); } } @@ -330,7 +325,7 @@ private static List copyEntries(List entriesToReturn, long startEn void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { + ReadEntriesContext context) { final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry); ConcurrentMap pendingReadsForLedger = @@ -345,8 +340,8 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp if (findBestCandidateOutcome.needsAdditionalReads()) { CompletableFuture> readFromMidFuture = new CompletableFuture<>(); - ReadEntriesCallback presentReadCallback = new ReadEntriesCallback(readFromMidFuture); - listenerAdded = pendingRead.addListener(presentReadCallback, ctx, key.startEntry, key.endEntry); + listenerAdded = pendingRead.addListener(ReadEntriesContext.fromFuture(readFromMidFuture), + key.startEntry, key.endEntry); if (!listenerAdded) { continue; } @@ -367,16 +362,16 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp }) .whenComplete((finalResult, e) -> { if (e != null) { - callback.readEntriesFailed(createManagedLedgerException(e), ctx); + context.fail(e); releaseEntriesSafely(readFromLeftFuture); releaseEntriesSafely(readFromMidFuture); releaseEntriesSafely(readFromRightFuture); } else { - callback.readEntriesComplete(finalResult, ctx); + context.complete(finalResult); } }); } else { - listenerAdded = pendingRead.addListener(callback, ctx, key.startEntry, key.endEntry); + listenerAdded = pendingRead.addListener(context, key.startEntry, key.endEntry); } if (createdByThisThread.get()) { @@ -393,9 +388,8 @@ private CompletableFuture> recursiveReadMissingEntriesAsync(ReadHand CompletableFuture> future; if (missingKey != null) { future = new CompletableFuture<>(); - ReadEntriesCallback callback = new ReadEntriesCallback(future); - rangeEntryCache.asyncReadEntry0(lh, missingKey.startEntry, missingKey.endEntry, expectedReadCount, callback, - null, false); + rangeEntryCache.asyncReadEntry0(lh, missingKey.startEntry, missingKey.endEntry, expectedReadCount, + ReadEntriesContext.fromFuture(future), false); } else { future = CompletableFuture.completedFuture(Collections.emptyList()); } @@ -417,22 +411,4 @@ void invalidateLedger(long id) { cachedPendingReads.remove(id); } - static class ReadEntriesCallback implements AsyncCallbacks.ReadEntriesCallback { - - private final CompletableFuture> completableFuture; - - public ReadEntriesCallback(CompletableFuture> completableFuture) { - this.completableFuture = completableFuture; - } - - @Override - public void readEntriesComplete(List entries, Object ctx) { - completableFuture.complete(entries); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - completableFuture.completeExceptionally(exception); - } - } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index fd391ba2bf6b0..a30bd89a10716 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -43,6 +43,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.ReadEntriesContext; import org.apache.bookkeeper.mledger.ReferenceCountedEntry; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; @@ -235,7 +236,7 @@ public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCall try { asyncReadEntriesByPosition(lh, position, position, 1, () -> DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY ? 1 : 0, - new ReadEntriesCallback() { + new ReadEntriesContext(new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { if (entries.isEmpty()) { @@ -249,7 +250,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { callback.readEntryFailed(exception, ctx); } - }, ctx, true); + }, ctx), true); } catch (Throwable t) { log.warn("failed to read entries for {}-{}", lh.getId(), position, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt @@ -264,7 +265,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, final ReadEntriesCallback callback, Object ctx) { try { - asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx, true); + asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, new ReadEntriesContext(callback, ctx), true); } catch (Throwable t) { log.warn("failed to read entries for {}--{}-{}", lh.getId(), firstEntry, lastEntry, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt @@ -277,18 +278,17 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSu @SuppressWarnings({ "unchecked", "rawtypes" }) void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - final ReadEntriesCallback callback, Object ctx, boolean acquirePermits) { + ReadEntriesContext context, boolean acquirePermits) { final long ledgerId = lh.getId(); final int numberOfEntries = (int) (lastEntry - firstEntry) + 1; final Position firstPosition = PositionFactory.create(ledgerId, firstEntry); final Position lastPosition = PositionFactory.create(ledgerId, lastEntry); - asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, callback, ctx, + asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, context, acquirePermits); } void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, - IntSupplier expectedReadCount, final ReadEntriesCallback originalCallback, - Object ctx, boolean acquirePermits) { + IntSupplier expectedReadCount, ReadEntriesContext context, boolean acquirePermits) { checkArgument(firstPosition.getLedgerId() == lastPosition.getLedgerId(), "Invalid range. Entries %s and %s should be in the same ledger.", firstPosition, lastPosition); @@ -303,8 +303,7 @@ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); if (!acquirePermits || pendingReadsLimiter.isDisabled()) { - doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, - originalCallback, ctx); + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, context); } else { long estimatedEntrySize = getEstimatedEntrySize(lh); long estimatedReadSize = numberOfEntries * estimatedEntrySize; @@ -319,20 +318,20 @@ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position // or timeout ml.getExecutor().execute(() -> { doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount, originalCallback, ctx, handle, estimatedReadSize); + expectedReadCount, context, handle, estimatedReadSize); }); }); // permits were immediately available and acquired if (optionalHandle.isPresent()) { doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount, originalCallback, ctx, optionalHandle.get(), estimatedReadSize); + expectedReadCount, context, optionalHandle.get(), estimatedReadSize); } } } void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, IntSupplier expectedReadCount, - final ReadEntriesCallback originalCallback, Object ctx, + ReadEntriesContext context, InflightReadsLimiter.Handle handle, long estimatedReadSize) { if (!handle.success()) { String message = String.format( @@ -342,42 +341,30 @@ void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition + "managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis and " + "managedLedgerMaxReadsInFlightSizeInMB)", lh.getId(), getName(), estimatedReadSize, numberOfEntries); - originalCallback.readEntriesFailed(new ManagedLedgerException.TooManyRequestsException(message), ctx); + context.fail(new ManagedLedgerException.TooManyRequestsException(message)); return; } InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); - ReadEntriesCallback wrappedCallback = new ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx2) { - if (!entries.isEmpty()) { - // release permits only when entries have been handled - AtomicInteger remainingCount = new AtomicInteger(entries.size()); - for (Entry entry : entries) { - ((EntryImpl) entry).onDeallocate(() -> { - if (remainingCount.decrementAndGet() <= 0) { - pendingReadsLimiter.release(handle); - } - }); - } - } else { - pendingReadsLimiter.release(handle); - } - originalCallback.readEntriesComplete(entries, ctx2); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) { + context.setBeforeCompleteCallback(entries -> { + if (entries.isEmpty()) { pendingReadsLimiter.release(handle); - originalCallback.readEntriesFailed(exception, ctx2); + } else { + // release permits only when entries have been handled + AtomicInteger remainingCount = new AtomicInteger(entries.size()); + for (Entry entry : entries) { + ((EntryImpl) entry).onDeallocate(() -> { + if (remainingCount.decrementAndGet() <= 0) { + pendingReadsLimiter.release(handle); + } + }); + } } - }; - doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, - wrappedCallback, ctx); + }); + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, context); } void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, - IntSupplier expectedReadCount, final ReadEntriesCallback callback, - Object ctx) { + IntSupplier expectedReadCount, ReadEntriesContext context) { Collection cachedEntries; if (firstPosition.compareTo(lastPosition) == 0) { ReferenceCountedEntry cachedEntry = entries.get(firstPosition); @@ -411,7 +398,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } if (cachedEntries.size() == numberOfEntries) { - callback.readEntriesComplete(entriesToReturn, ctx); + context.complete(entriesToReturn); } else { // read missing ranges long firstEntryInRange = -1; @@ -425,7 +412,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio if (firstEntryInRange != -1) { futures.add( readMissingEntriesAsync(lh, firstEntryInRange, firstPosition.getEntryId() + i - 1, - expectedReadCount, ctx)); + expectedReadCount)); firstEntryInRange = -1; } } @@ -433,7 +420,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio if (firstEntryInRange != -1) { futures.add( readMissingEntriesAsync(lh, firstEntryInRange, lastPosition.getEntryId(), - expectedReadCount, ctx)); + expectedReadCount)); } FutureUtil.waitForAll(futures).whenComplete((__, t) -> { if (t != null) { @@ -457,7 +444,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio log.warn("Failed to read missing entries from bookkeeper, retrying by reading all", t); // Read all the entries from bookkeeper pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), - expectedReadCount, callback, ctx); + expectedReadCount, context); return; } for (CompletableFuture> future : futures) { @@ -474,22 +461,21 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } } } - callback.readEntriesComplete(entriesToReturn, ctx); + context.complete(entriesToReturn); }); } } else { // Read all the entries from bookkeeper pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), - expectedReadCount, callback, ctx); + expectedReadCount, context); } } - private CompletableFuture> readMissingEntriesAsync(ReadHandle lh, - long firstEntry, long lastEntry, - IntSupplier expectedReadCount, Object ctx) { + private CompletableFuture> readMissingEntriesAsync( + ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount) { CompletableFuture> future = new CompletableFuture<>(); - PendingReadsManager.ReadEntriesCallback callback = new PendingReadsManager.ReadEntriesCallback(future); - pendingReadsManager.readEntries(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx); + pendingReadsManager.readEntries(lh, firstEntry, lastEntry, expectedReadCount, + ReadEntriesContext.fromFuture(future)); return future; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 9c6c79eb95b40..a4eaac5c491f6 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -54,6 +54,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.ReadEntriesContext; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; @@ -115,13 +116,12 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { long startEntry = invocationOnMock.getArgument(1); long endEntry = invocationOnMock.getArgument(2); IntSupplier expectedReadCount = invocationOnMock.getArgument(3); - AsyncCallbacks.ReadEntriesCallback callback = invocationOnMock.getArgument(4); - Object ctx = invocationOnMock.getArgument(5); - pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, callback, ctx); + ReadEntriesContext context = invocationOnMock.getArgument(4); + pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, context); return null; } }).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(), - any(), any(), any(), anyBoolean()); + any(), any(), anyBoolean()); lh = mock(ReadHandle.class); ml = mock(ManagedLedgerImpl.class); From 0195041453fc59ca6297672ead838abee9558789 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 14:23:06 +0800 Subject: [PATCH 02/16] Revert "Introduce ReadEntriesContext to ensure callback is only called once" This reverts commit b9449e61e58c2356e631fe1a65b0920b3c209784. --- .../mledger/ReadEntriesContext.java | 90 ------------------- .../impl/cache/PendingReadsManager.java | 58 ++++++++---- .../impl/cache/RangeEntryCacheImpl.java | 88 ++++++++++-------- .../impl/cache/PendingReadsManagerTest.java | 8 +- 4 files changed, 96 insertions(+), 148 deletions(-) delete mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java deleted file mode 100644 index d0621b422388a..0000000000000 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ReadEntriesContext.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.bookkeeper.mledger; - -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.function.Consumer; - -/** - * It wraps the read entries' callback and context to ensure that the callback can only be triggered once. - */ -public class ReadEntriesContext { - - private AsyncCallbacks.ReadEntriesCallback callback; - private final Object ctx; - private Consumer> beforeCompleteCallback = null; - - public ReadEntriesContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { - this.callback = callback; - this.ctx = ctx; - } - - public void complete(List entries) { - AsyncCallbacks.ReadEntriesCallback callback; - Consumer> beforeCompleteCallback; - synchronized (this) { - beforeCompleteCallback = this.beforeCompleteCallback; - this.beforeCompleteCallback = null; - callback = this.callback; - this.callback = null; - } - if (beforeCompleteCallback != null) { - beforeCompleteCallback.accept(entries); - } - if (callback != null) { - callback.readEntriesComplete(entries, ctx); - } - } - - public void fail(Throwable throwable) { - AsyncCallbacks.ReadEntriesCallback callback; - Consumer> beforeCompleteCallback; - synchronized (this) { - beforeCompleteCallback = this.beforeCompleteCallback; - this.beforeCompleteCallback = null; - callback = this.callback; - this.callback = null; - } - if (beforeCompleteCallback != null) { - beforeCompleteCallback.accept(List.of()); - } - if (callback != null) { - callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); - } - } - - public synchronized void setBeforeCompleteCallback(Consumer> beforeCompleteCallback) { - this.beforeCompleteCallback = beforeCompleteCallback; - } - - public static ReadEntriesContext fromFuture(CompletableFuture> future) { - return new ReadEntriesContext(new AsyncCallbacks.ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx) { - future.complete(entries); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(exception); - } - }, null); - } -} diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index a956cf61fb913..5f904f3cf8420 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl.cache; +import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; import io.prometheus.client.Counter; import java.util.ArrayList; import java.util.Collections; @@ -30,8 +31,9 @@ import java.util.function.IntSupplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ReadEntriesContext; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.EntryImpl; /** @@ -130,7 +132,7 @@ PendingReadKey reminderOnRight(PendingReadKey other) { } - private record ReadEntriesCallbackWithContext(ReadEntriesContext readEntriesContext, + private record ReadEntriesCallbackWithContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx, long startEntry, long endEntry) { } @@ -246,11 +248,12 @@ public synchronized void attach(CompletableFuture> handle) { }); } - synchronized boolean addListener(ReadEntriesContext context, long startEntry, long endEntry) { + synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback, + Object ctx, long startEntry, long endEntry) { if (state == PendingReadState.COMPLETED) { return false; } - listeners.add(new ReadEntriesCallbackWithContext(context, startEntry, endEntry)); + listeners.add(new ReadEntriesCallbackWithContext(callback, ctx, startEntry, endEntry)); return true; } @@ -273,14 +276,15 @@ private void readEntriesComplete(List callbacks, if (first.startEntry == key.startEntry && first.endEntry == key.endEntry) { // perfect match, no copy, this is the most common case - first.readEntriesContext.complete(entriesToReturn); + first.callback.readEntriesComplete(entriesToReturn, first.ctx); } else { - first.readEntriesContext.complete(keepEntries(entriesToReturn, first.startEntry, first.endEntry)); + first.callback.readEntriesComplete( + keepEntries(entriesToReturn, first.startEntry, first.endEntry), first.ctx); } } else { for (ReadEntriesCallbackWithContext callback : callbacks) { - callback.readEntriesContext.complete( - copyEntries(entriesToReturn, callback.startEntry, callback.endEntry)); + callback.callback.readEntriesComplete( + copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), callback.ctx); } for (Entry entry : entriesToReturn) { // don't decrease the read count when these entries are released @@ -293,7 +297,8 @@ private void readEntriesComplete(List callbacks, // this method isn't synchronized since that could lead to deadlocks private void readEntriesFailed(List callbacks, Throwable error) { for (ReadEntriesCallbackWithContext callback : callbacks) { - callback.readEntriesContext.fail(error); + ManagedLedgerException mlException = createManagedLedgerException(error); + callback.callback.readEntriesFailed(mlException, callback.ctx); } } @@ -325,7 +330,7 @@ private static List copyEntries(List entriesToReturn, long startEn void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - ReadEntriesContext context) { + final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry); ConcurrentMap pendingReadsForLedger = @@ -340,8 +345,8 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp if (findBestCandidateOutcome.needsAdditionalReads()) { CompletableFuture> readFromMidFuture = new CompletableFuture<>(); - listenerAdded = pendingRead.addListener(ReadEntriesContext.fromFuture(readFromMidFuture), - key.startEntry, key.endEntry); + ReadEntriesCallback presentReadCallback = new ReadEntriesCallback(readFromMidFuture); + listenerAdded = pendingRead.addListener(presentReadCallback, ctx, key.startEntry, key.endEntry); if (!listenerAdded) { continue; } @@ -362,16 +367,16 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp }) .whenComplete((finalResult, e) -> { if (e != null) { - context.fail(e); + callback.readEntriesFailed(createManagedLedgerException(e), ctx); releaseEntriesSafely(readFromLeftFuture); releaseEntriesSafely(readFromMidFuture); releaseEntriesSafely(readFromRightFuture); } else { - context.complete(finalResult); + callback.readEntriesComplete(finalResult, ctx); } }); } else { - listenerAdded = pendingRead.addListener(context, key.startEntry, key.endEntry); + listenerAdded = pendingRead.addListener(callback, ctx, key.startEntry, key.endEntry); } if (createdByThisThread.get()) { @@ -388,8 +393,9 @@ private CompletableFuture> recursiveReadMissingEntriesAsync(ReadHand CompletableFuture> future; if (missingKey != null) { future = new CompletableFuture<>(); - rangeEntryCache.asyncReadEntry0(lh, missingKey.startEntry, missingKey.endEntry, expectedReadCount, - ReadEntriesContext.fromFuture(future), false); + ReadEntriesCallback callback = new ReadEntriesCallback(future); + rangeEntryCache.asyncReadEntry0(lh, missingKey.startEntry, missingKey.endEntry, expectedReadCount, callback, + null, false); } else { future = CompletableFuture.completedFuture(Collections.emptyList()); } @@ -411,4 +417,22 @@ void invalidateLedger(long id) { cachedPendingReads.remove(id); } + static class ReadEntriesCallback implements AsyncCallbacks.ReadEntriesCallback { + + private final CompletableFuture> completableFuture; + + public ReadEntriesCallback(CompletableFuture> completableFuture) { + this.completableFuture = completableFuture; + } + + @Override + public void readEntriesComplete(List entries, Object ctx) { + completableFuture.complete(entries); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + completableFuture.completeExceptionally(exception); + } + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index a30bd89a10716..fd391ba2bf6b0 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -43,7 +43,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.ReadEntriesContext; import org.apache.bookkeeper.mledger.ReferenceCountedEntry; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; @@ -236,7 +235,7 @@ public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCall try { asyncReadEntriesByPosition(lh, position, position, 1, () -> DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY ? 1 : 0, - new ReadEntriesContext(new ReadEntriesCallback() { + new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { if (entries.isEmpty()) { @@ -250,7 +249,7 @@ public void readEntriesComplete(List entries, Object ctx) { public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { callback.readEntryFailed(exception, ctx); } - }, ctx), true); + }, ctx, true); } catch (Throwable t) { log.warn("failed to read entries for {}-{}", lh.getId(), position, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt @@ -265,7 +264,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, final ReadEntriesCallback callback, Object ctx) { try { - asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, new ReadEntriesContext(callback, ctx), true); + asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx, true); } catch (Throwable t) { log.warn("failed to read entries for {}--{}-{}", lh.getId(), firstEntry, lastEntry, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt @@ -278,17 +277,18 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSu @SuppressWarnings({ "unchecked", "rawtypes" }) void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - ReadEntriesContext context, boolean acquirePermits) { + final ReadEntriesCallback callback, Object ctx, boolean acquirePermits) { final long ledgerId = lh.getId(); final int numberOfEntries = (int) (lastEntry - firstEntry) + 1; final Position firstPosition = PositionFactory.create(ledgerId, firstEntry); final Position lastPosition = PositionFactory.create(ledgerId, lastEntry); - asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, context, + asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, callback, ctx, acquirePermits); } void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, - IntSupplier expectedReadCount, ReadEntriesContext context, boolean acquirePermits) { + IntSupplier expectedReadCount, final ReadEntriesCallback originalCallback, + Object ctx, boolean acquirePermits) { checkArgument(firstPosition.getLedgerId() == lastPosition.getLedgerId(), "Invalid range. Entries %s and %s should be in the same ledger.", firstPosition, lastPosition); @@ -303,7 +303,8 @@ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); if (!acquirePermits || pendingReadsLimiter.isDisabled()) { - doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, context); + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, + originalCallback, ctx); } else { long estimatedEntrySize = getEstimatedEntrySize(lh); long estimatedReadSize = numberOfEntries * estimatedEntrySize; @@ -318,20 +319,20 @@ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position // or timeout ml.getExecutor().execute(() -> { doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount, context, handle, estimatedReadSize); + expectedReadCount, originalCallback, ctx, handle, estimatedReadSize); }); }); // permits were immediately available and acquired if (optionalHandle.isPresent()) { doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount, context, optionalHandle.get(), estimatedReadSize); + expectedReadCount, originalCallback, ctx, optionalHandle.get(), estimatedReadSize); } } } void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, IntSupplier expectedReadCount, - ReadEntriesContext context, + final ReadEntriesCallback originalCallback, Object ctx, InflightReadsLimiter.Handle handle, long estimatedReadSize) { if (!handle.success()) { String message = String.format( @@ -341,30 +342,42 @@ void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition + "managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis and " + "managedLedgerMaxReadsInFlightSizeInMB)", lh.getId(), getName(), estimatedReadSize, numberOfEntries); - context.fail(new ManagedLedgerException.TooManyRequestsException(message)); + originalCallback.readEntriesFailed(new ManagedLedgerException.TooManyRequestsException(message), ctx); return; } InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); - context.setBeforeCompleteCallback(entries -> { - if (entries.isEmpty()) { - pendingReadsLimiter.release(handle); - } else { - // release permits only when entries have been handled - AtomicInteger remainingCount = new AtomicInteger(entries.size()); - for (Entry entry : entries) { - ((EntryImpl) entry).onDeallocate(() -> { - if (remainingCount.decrementAndGet() <= 0) { - pendingReadsLimiter.release(handle); - } - }); + ReadEntriesCallback wrappedCallback = new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx2) { + if (!entries.isEmpty()) { + // release permits only when entries have been handled + AtomicInteger remainingCount = new AtomicInteger(entries.size()); + for (Entry entry : entries) { + ((EntryImpl) entry).onDeallocate(() -> { + if (remainingCount.decrementAndGet() <= 0) { + pendingReadsLimiter.release(handle); + } + }); + } + } else { + pendingReadsLimiter.release(handle); } + originalCallback.readEntriesComplete(entries, ctx2); } - }); - doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, context); + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) { + pendingReadsLimiter.release(handle); + originalCallback.readEntriesFailed(exception, ctx2); + } + }; + doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, + wrappedCallback, ctx); } void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, - IntSupplier expectedReadCount, ReadEntriesContext context) { + IntSupplier expectedReadCount, final ReadEntriesCallback callback, + Object ctx) { Collection cachedEntries; if (firstPosition.compareTo(lastPosition) == 0) { ReferenceCountedEntry cachedEntry = entries.get(firstPosition); @@ -398,7 +411,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } if (cachedEntries.size() == numberOfEntries) { - context.complete(entriesToReturn); + callback.readEntriesComplete(entriesToReturn, ctx); } else { // read missing ranges long firstEntryInRange = -1; @@ -412,7 +425,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio if (firstEntryInRange != -1) { futures.add( readMissingEntriesAsync(lh, firstEntryInRange, firstPosition.getEntryId() + i - 1, - expectedReadCount)); + expectedReadCount, ctx)); firstEntryInRange = -1; } } @@ -420,7 +433,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio if (firstEntryInRange != -1) { futures.add( readMissingEntriesAsync(lh, firstEntryInRange, lastPosition.getEntryId(), - expectedReadCount)); + expectedReadCount, ctx)); } FutureUtil.waitForAll(futures).whenComplete((__, t) -> { if (t != null) { @@ -444,7 +457,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio log.warn("Failed to read missing entries from bookkeeper, retrying by reading all", t); // Read all the entries from bookkeeper pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), - expectedReadCount, context); + expectedReadCount, callback, ctx); return; } for (CompletableFuture> future : futures) { @@ -461,21 +474,22 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } } } - context.complete(entriesToReturn); + callback.readEntriesComplete(entriesToReturn, ctx); }); } } else { // Read all the entries from bookkeeper pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), - expectedReadCount, context); + expectedReadCount, callback, ctx); } } - private CompletableFuture> readMissingEntriesAsync( - ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount) { + private CompletableFuture> readMissingEntriesAsync(ReadHandle lh, + long firstEntry, long lastEntry, + IntSupplier expectedReadCount, Object ctx) { CompletableFuture> future = new CompletableFuture<>(); - pendingReadsManager.readEntries(lh, firstEntry, lastEntry, expectedReadCount, - ReadEntriesContext.fromFuture(future)); + PendingReadsManager.ReadEntriesCallback callback = new PendingReadsManager.ReadEntriesCallback(future); + pendingReadsManager.readEntries(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx); return future; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index a4eaac5c491f6..9c6c79eb95b40 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -54,7 +54,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.ReadEntriesContext; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; @@ -116,12 +115,13 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { long startEntry = invocationOnMock.getArgument(1); long endEntry = invocationOnMock.getArgument(2); IntSupplier expectedReadCount = invocationOnMock.getArgument(3); - ReadEntriesContext context = invocationOnMock.getArgument(4); - pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, context); + AsyncCallbacks.ReadEntriesCallback callback = invocationOnMock.getArgument(4); + Object ctx = invocationOnMock.getArgument(5); + pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, callback, ctx); return null; } }).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(), - any(), any(), anyBoolean()); + any(), any(), any(), anyBoolean()); lh = mock(ReadHandle.class); ml = mock(ManagedLedgerImpl.class); From f99b2ea434b6b51cb1e497d67b0b50a53d319c63 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 15:22:36 +0800 Subject: [PATCH 03/16] Step 1: use future-based APIs --- .../mledger/impl/cache/EntryCache.java | 20 +-- .../impl/cache/EntryCacheDisabled.java | 27 ++- .../impl/cache/PendingReadsManager.java | 67 +++---- .../impl/cache/RangeEntryCacheImpl.java | 169 ++++++++---------- .../mledger/util/CallbackUtils.java | 60 +++++++ .../impl/cache/PendingReadsManagerTest.java | 7 +- 6 files changed, 179 insertions(+), 171 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java index b2ebf7430560c..775812c26dc0f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCache.java @@ -18,10 +18,10 @@ */ package org.apache.bookkeeper.mledger.impl.cache; +import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.function.IntSupplier; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; @@ -81,13 +81,10 @@ public interface EntryCache { * the last entry to read (inclusive) * @param expectedReadCount resolves the expected read count for the given entry. When the expected read count is * >0, the entry can be cached and reused later. - * @param callback - * the callback object that will be notified when read is done - * @param ctx - * the context object + * @return the future of entries */ - void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - ReadEntriesCallback callback, Object ctx); + CompletableFuture> asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, + IntSupplier expectedReadCount); /** * Read entry at given position from the cache or from bookkeeper. @@ -98,12 +95,9 @@ void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier * the ledger handle * @param position * position to read the entry from - * @param callback - * the callback object that will be notified when read is done - * @param ctx - * the context object + * @return the future of the entry */ - void asyncReadEntry(ReadHandle lh, Position position, ReadEntryCallback callback, Object ctx); + CompletableFuture asyncReadEntry(ReadHandle lh, Position position); /** * Get the total size in bytes of all the entries stored in this cache. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java index b5a45415a4fe1..a4e26fb2923f7 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java @@ -22,10 +22,10 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.function.IntSupplier; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; @@ -68,9 +68,9 @@ public void clear() { } @Override - public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { - ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry).thenAcceptAsync( + public CompletableFuture> asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, + IntSupplier expectedReadCount) { + return ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry).thenApplyAsync( ledgerEntries -> { List entries = new ArrayList<>(); long totalSize = 0; @@ -89,21 +89,18 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSu ml.getFactory().getMbean().recordCacheMiss(entries.size(), totalSize); ml.getMbean().addReadEntriesSample(entries.size(), totalSize); - callback.readEntriesComplete(entries, ctx); - }, ml.getExecutor()).exceptionally(exception -> { - callback.readEntriesFailed(createManagedLedgerException(exception), ctx); - return null; - }); + return entries; + }, ml.getExecutor()); } @Override - public void asyncReadEntry(ReadHandle lh, Position position, AsyncCallbacks.ReadEntryCallback callback, - Object ctx) { + public CompletableFuture asyncReadEntry(ReadHandle lh, Position position) { + final var future = new CompletableFuture(); ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).whenCompleteAsync( (ledgerEntries, exception) -> { if (exception != null) { ml.invalidateLedgerHandle(lh); - callback.readEntryFailed(createManagedLedgerException(exception), ctx); + future.completeExceptionally(createManagedLedgerException(exception)); return; } @@ -116,15 +113,15 @@ public void asyncReadEntry(ReadHandle lh, Position position, AsyncCallbacks.Read ml.getMbean().recordReadEntriesOpsCacheMisses(1, returnEntry.getLength()); ml.getFactory().getMbean().recordCacheMiss(1, returnEntry.getLength()); ml.getMbean().addReadEntriesSample(1, returnEntry.getLength()); - callback.readEntryComplete(returnEntry, ctx); + future.complete(returnEntry); } else { - callback.readEntryFailed(new ManagedLedgerException("Could not read given position"), - ctx); + future.completeExceptionally(new ManagedLedgerException("Could not read given position")); } } finally { ledgerEntries.close(); } }, ml.getExecutor()); + return future; } @Override diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index 5f904f3cf8420..1d5a6cc74d066 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -18,7 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl.cache; -import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; +import com.google.common.annotations.VisibleForTesting; import io.prometheus.client.Counter; import java.util.ArrayList; import java.util.Collections; @@ -31,9 +31,7 @@ import java.util.function.IntSupplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.EntryImpl; /** @@ -132,7 +130,7 @@ PendingReadKey reminderOnRight(PendingReadKey other) { } - private record ReadEntriesCallbackWithContext(AsyncCallbacks.ReadEntriesCallback callback, Object ctx, + private record ReadEntriesCallbackWithContext(CompletableFuture> future, long startEntry, long endEntry) { } @@ -248,12 +246,11 @@ public synchronized void attach(CompletableFuture> handle) { }); } - synchronized boolean addListener(AsyncCallbacks.ReadEntriesCallback callback, - Object ctx, long startEntry, long endEntry) { + synchronized boolean addListener(CompletableFuture> entriesFuture, long startEntry, long endEntry) { if (state == PendingReadState.COMPLETED) { return false; } - listeners.add(new ReadEntriesCallbackWithContext(callback, ctx, startEntry, endEntry)); + listeners.add(new ReadEntriesCallbackWithContext(entriesFuture, startEntry, endEntry)); return true; } @@ -276,15 +273,13 @@ private void readEntriesComplete(List callbacks, if (first.startEntry == key.startEntry && first.endEntry == key.endEntry) { // perfect match, no copy, this is the most common case - first.callback.readEntriesComplete(entriesToReturn, first.ctx); + first.future.complete(entriesToReturn); } else { - first.callback.readEntriesComplete( - keepEntries(entriesToReturn, first.startEntry, first.endEntry), first.ctx); + first.future.complete(keepEntries(entriesToReturn, first.startEntry, first.endEntry)); } } else { for (ReadEntriesCallbackWithContext callback : callbacks) { - callback.callback.readEntriesComplete( - copyEntries(entriesToReturn, callback.startEntry, callback.endEntry), callback.ctx); + callback.future.complete(copyEntries(entriesToReturn, callback.startEntry, callback.endEntry)); } for (Entry entry : entriesToReturn) { // don't decrease the read count when these entries are released @@ -297,8 +292,7 @@ private void readEntriesComplete(List callbacks, // this method isn't synchronized since that could lead to deadlocks private void readEntriesFailed(List callbacks, Throwable error) { for (ReadEntriesCallbackWithContext callback : callbacks) { - ManagedLedgerException mlException = createManagedLedgerException(error); - callback.callback.readEntriesFailed(mlException, callback.ctx); + callback.future.completeExceptionally(error); } } @@ -328,9 +322,10 @@ private static List copyEntries(List entriesToReturn, long startEn } } - - void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { + @VisibleForTesting + CompletableFuture> readEntries(ReadHandle lh, long firstEntry, long lastEntry, + IntSupplier expectedReadCount) { + final var readFuture = new CompletableFuture>(); final PendingReadKey key = new PendingReadKey(firstEntry, lastEntry); ConcurrentMap pendingReadsForLedger = @@ -345,8 +340,7 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp if (findBestCandidateOutcome.needsAdditionalReads()) { CompletableFuture> readFromMidFuture = new CompletableFuture<>(); - ReadEntriesCallback presentReadCallback = new ReadEntriesCallback(readFromMidFuture); - listenerAdded = pendingRead.addListener(presentReadCallback, ctx, key.startEntry, key.endEntry); + listenerAdded = pendingRead.addListener(readFromMidFuture, key.startEntry, key.endEntry); if (!listenerAdded) { continue; } @@ -367,16 +361,16 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp }) .whenComplete((finalResult, e) -> { if (e != null) { - callback.readEntriesFailed(createManagedLedgerException(e), ctx); + readFuture.completeExceptionally(e); releaseEntriesSafely(readFromLeftFuture); releaseEntriesSafely(readFromMidFuture); releaseEntriesSafely(readFromRightFuture); } else { - callback.readEntriesComplete(finalResult, ctx); + readFuture.complete(finalResult); } }); } else { - listenerAdded = pendingRead.addListener(callback, ctx, key.startEntry, key.endEntry); + listenerAdded = pendingRead.addListener(readFuture, key.startEntry, key.endEntry); } if (createdByThisThread.get()) { @@ -385,21 +379,18 @@ void readEntries(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier exp pendingRead.attach(readResult); } } + return readFuture; } private CompletableFuture> recursiveReadMissingEntriesAsync(ReadHandle lh, IntSupplier expectedReadCount, PendingReadKey missingKey) { - CompletableFuture> future; if (missingKey != null) { - future = new CompletableFuture<>(); - ReadEntriesCallback callback = new ReadEntriesCallback(future); - rangeEntryCache.asyncReadEntry0(lh, missingKey.startEntry, missingKey.endEntry, expectedReadCount, callback, - null, false); + return rangeEntryCache.asyncReadEntry0(lh, missingKey.startEntry, missingKey.endEntry, expectedReadCount, + false); } else { - future = CompletableFuture.completedFuture(Collections.emptyList()); + return CompletableFuture.completedFuture(Collections.emptyList()); } - return future; } private void releaseEntriesSafely(CompletableFuture> future) { @@ -417,22 +408,4 @@ void invalidateLedger(long id) { cachedPendingReads.remove(id); } - static class ReadEntriesCallback implements AsyncCallbacks.ReadEntriesCallback { - - private final CompletableFuture> completableFuture; - - public ReadEntriesCallback(CompletableFuture> completableFuture) { - this.completableFuture = completableFuture; - } - - @Override - public void readEntriesComplete(List entries, Object ctx) { - completableFuture.complete(entries); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - completableFuture.completeExceptionally(exception); - } - } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index fd391ba2bf6b0..887ccf310f438 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -36,8 +36,6 @@ import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; @@ -230,65 +228,63 @@ public void invalidateAllEntries(long ledgerId) { } @Override - public void asyncReadEntry(ReadHandle lh, Position position, final ReadEntryCallback callback, - final Object ctx) { + public CompletableFuture asyncReadEntry(ReadHandle lh, Position position) { try { + final var future = new CompletableFuture(); asyncReadEntriesByPosition(lh, position, position, 1, - () -> DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY ? 1 : 0, - new ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx) { + () -> DEFAULT_CACHE_INDIVIDUAL_READ_ENTRY ? 1 : 0, true + ).whenComplete((entries, throwable) -> { + if (throwable == null) { if (entries.isEmpty()) { - callback.readEntryFailed(new ManagedLedgerException("Could not read given position"), ctx); + future.completeExceptionally(new ManagedLedgerException("Could not read given position")); } else { - callback.readEntryComplete(entries.get(0), ctx); + future.complete(entries.get(0)); } + } else { + future.completeExceptionally(ManagedLedgerException.getManagedLedgerException(throwable)); } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - callback.readEntryFailed(exception, ctx); - } - }, ctx, true); + }); + return future; } catch (Throwable t) { log.warn("failed to read entries for {}-{}", lh.getId(), position, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt // (entry.data is already deallocate due to any race-condition) so, invalidate cache and next time read from // the bookie) invalidateAllEntries(lh.getId()); - callback.readEntryFailed(createManagedLedgerException(t), ctx); + return CompletableFuture.failedFuture(createManagedLedgerException(t)); } } @Override - public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - final ReadEntriesCallback callback, Object ctx) { + public CompletableFuture> asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, + IntSupplier expectedReadCount) { try { - asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx, true); + return asyncReadEntry0(lh, firstEntry, lastEntry, expectedReadCount, true); } catch (Throwable t) { log.warn("failed to read entries for {}--{}-{}", lh.getId(), firstEntry, lastEntry, t); // invalidate all entries related to ledger from the cache (it might happen if entry gets corrupt // (entry.data is already deallocate due to any race-condition) so, invalidate cache and next time read from // the bookie) invalidateAllEntries(lh.getId()); - callback.readEntriesFailed(createManagedLedgerException(t), ctx); + return CompletableFuture.failedFuture(t); } } @SuppressWarnings({ "unchecked", "rawtypes" }) - void asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, - final ReadEntriesCallback callback, Object ctx, boolean acquirePermits) { + CompletableFuture> asyncReadEntry0(ReadHandle lh, long firstEntry, long lastEntry, + IntSupplier expectedReadCount, + boolean acquirePermits) { final long ledgerId = lh.getId(); final int numberOfEntries = (int) (lastEntry - firstEntry) + 1; final Position firstPosition = PositionFactory.create(ledgerId, firstEntry); final Position lastPosition = PositionFactory.create(ledgerId, lastEntry); - asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, callback, ctx, + return asyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, acquirePermits); } - void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, - IntSupplier expectedReadCount, final ReadEntriesCallback originalCallback, - Object ctx, boolean acquirePermits) { + CompletableFuture> asyncReadEntriesByPosition( + ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, + IntSupplier expectedReadCount, boolean acquirePermits) { checkArgument(firstPosition.getLedgerId() == lastPosition.getLedgerId(), "Invalid range. Entries %s and %s should be in the same ledger.", firstPosition, lastPosition); @@ -303,8 +299,7 @@ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); if (!acquirePermits || pendingReadsLimiter.isDisabled()) { - doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, - originalCallback, ctx); + return doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount); } else { long estimatedEntrySize = getEstimatedEntrySize(lh); long estimatedReadSize = numberOfEntries * estimatedEntrySize; @@ -313,27 +308,34 @@ void asyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position estimatedReadSize, numberOfEntries, estimatedEntrySize); } + final var future = new CompletableFuture>(); Optional optionalHandle = pendingReadsLimiter.acquire(estimatedReadSize, handle -> { // permits were not immediately available, callback will be executed when permits are acquired // or timeout - ml.getExecutor().execute(() -> { - doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount, originalCallback, ctx, handle, estimatedReadSize); - }); + ml.getExecutor().execute(() -> doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, + lastPosition, numberOfEntries, expectedReadCount, handle, estimatedReadSize + ).whenComplete((entries, e) -> { + if (e == null) { + future.complete(entries); + } else { + future.completeExceptionally(e); + } + })); }); // permits were immediately available and acquired if (optionalHandle.isPresent()) { - doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount, originalCallback, ctx, optionalHandle.get(), estimatedReadSize); + return doAsyncReadEntriesWithAcquiredPermits(lh, firstPosition, lastPosition, numberOfEntries, + expectedReadCount, optionalHandle.get(), estimatedReadSize); + } else { + return future; // will be completed by `pendingReadsLimiter` } } } - void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition, Position lastPosition, - int numberOfEntries, IntSupplier expectedReadCount, - final ReadEntriesCallback originalCallback, Object ctx, - InflightReadsLimiter.Handle handle, long estimatedReadSize) { + CompletableFuture> doAsyncReadEntriesWithAcquiredPermits( + ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, + IntSupplier expectedReadCount, InflightReadsLimiter.Handle handle, long estimatedReadSize) { if (!handle.success()) { String message = String.format( "Couldn't acquire enough permits on the max reads in flight limiter to read from ledger " @@ -342,42 +344,35 @@ void doAsyncReadEntriesWithAcquiredPermits(ReadHandle lh, Position firstPosition + "managedLedgerMaxReadsInFlightPermitsAcquireTimeoutMillis and " + "managedLedgerMaxReadsInFlightSizeInMB)", lh.getId(), getName(), estimatedReadSize, numberOfEntries); - originalCallback.readEntriesFailed(new ManagedLedgerException.TooManyRequestsException(message), ctx); - return; + return CompletableFuture.failedFuture(new ManagedLedgerException.TooManyRequestsException(message)); } InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); - ReadEntriesCallback wrappedCallback = new ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx2) { - if (!entries.isEmpty()) { - // release permits only when entries have been handled - AtomicInteger remainingCount = new AtomicInteger(entries.size()); - for (Entry entry : entries) { - ((EntryImpl) entry).onDeallocate(() -> { - if (remainingCount.decrementAndGet() <= 0) { - pendingReadsLimiter.release(handle); - } - }); - } - } else { - pendingReadsLimiter.release(handle); + final var future = doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, + expectedReadCount); + future.thenAccept(entries -> { + if (!entries.isEmpty()) { + // release permits only when entries have been handled + AtomicInteger remainingCount = new AtomicInteger(entries.size()); + for (Entry entry : entries) { + ((EntryImpl) entry).onDeallocate(() -> { + if (remainingCount.decrementAndGet() <= 0) { + pendingReadsLimiter.release(handle); + } + }); } - originalCallback.readEntriesComplete(entries, ctx2); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx2) { + } else { pendingReadsLimiter.release(handle); - originalCallback.readEntriesFailed(exception, ctx2); } - }; - doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, expectedReadCount, - wrappedCallback, ctx); + }).exceptionally(e -> { + pendingReadsLimiter.release(handle); + return null; + }); + return future; } - void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, - IntSupplier expectedReadCount, final ReadEntriesCallback callback, - Object ctx) { + CompletableFuture> doAsyncReadEntriesByPosition( + ReadHandle lh, Position firstPosition, Position lastPosition, int numberOfEntries, + IntSupplier expectedReadCount) { Collection cachedEntries; if (firstPosition.compareTo(lastPosition) == 0) { ReferenceCountedEntry cachedEntry = entries.get(firstPosition); @@ -411,7 +406,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } if (cachedEntries.size() == numberOfEntries) { - callback.readEntriesComplete(entriesToReturn, ctx); + return CompletableFuture.completedFuture(entriesToReturn); } else { // read missing ranges long firstEntryInRange = -1; @@ -423,19 +418,17 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } } else { if (firstEntryInRange != -1) { - futures.add( - readMissingEntriesAsync(lh, firstEntryInRange, firstPosition.getEntryId() + i - 1, - expectedReadCount, ctx)); + futures.add(pendingReadsManager.readEntries(lh, firstEntryInRange, + firstPosition.getEntryId() + i - 1, expectedReadCount)); firstEntryInRange = -1; } } } if (firstEntryInRange != -1) { - futures.add( - readMissingEntriesAsync(lh, firstEntryInRange, lastPosition.getEntryId(), - expectedReadCount, ctx)); + futures.add(pendingReadsManager.readEntries(lh, firstEntryInRange, lastPosition.getEntryId(), + expectedReadCount)); } - FutureUtil.waitForAll(futures).whenComplete((__, t) -> { + return FutureUtil.waitForAll(futures).handle((__, t) -> { if (t != null) { // release cached entries placed in entriesToReturn for (Entry entry : entriesToReturn) { @@ -456,9 +449,7 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } log.warn("Failed to read missing entries from bookkeeper, retrying by reading all", t); // Read all the entries from bookkeeper - pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), - expectedReadCount, callback, ctx); - return; + return Optional.>empty(); } for (CompletableFuture> future : futures) { List readEntries = future.getNow(null); @@ -474,25 +465,19 @@ void doAsyncReadEntriesByPosition(ReadHandle lh, Position firstPosition, Positio } } } - callback.readEntriesComplete(entriesToReturn, ctx); - }); + return Optional.of(entriesToReturn); + }).thenCompose(optEntries -> optEntries + .map(CompletableFuture::completedFuture) + .orElseGet(() -> pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), + lastPosition.getEntryId(), expectedReadCount))); } } else { // Read all the entries from bookkeeper - pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), - expectedReadCount, callback, ctx); + return pendingReadsManager.readEntries(lh, firstPosition.getEntryId(), lastPosition.getEntryId(), + expectedReadCount); } } - private CompletableFuture> readMissingEntriesAsync(ReadHandle lh, - long firstEntry, long lastEntry, - IntSupplier expectedReadCount, Object ctx) { - CompletableFuture> future = new CompletableFuture<>(); - PendingReadsManager.ReadEntriesCallback callback = new PendingReadsManager.ReadEntriesCallback(future); - pendingReadsManager.readEntries(lh, firstEntry, lastEntry, expectedReadCount, callback, ctx); - return future; - } - @VisibleForTesting public long getEstimatedEntrySize(ReadHandle lh) { if (lh.getLength() == 0 || lh.getLastAddConfirmed() < 0) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java new file mode 100644 index 0000000000000..da10cb9fae1e6 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.util; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.AsyncCallbacks; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerException; + +/** + * Currently, the managed ledger APIs mostly accept a callback and a context that is passed to the callback, if the + * callback is called twice unexpectedly, the caller side might fail, e.g. the context is a recyclable object whose + * `recycle()` method is called when it's passed to the callback. + * This util class delegates the callback and the corresponding context object to a {@link CompletableFuture}, which + * guarantees the callback can be called exactly once in {@link CompletableFuture#whenComplete} when the future is + * completed. + */ +public class CallbackUtils { + + public static CompletableFuture> wrap(AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { + final var future = new CompletableFuture>(); + future.whenComplete((entries, throwable) -> { + if (throwable == null) { + callback.readEntriesComplete(entries, ctx); + } else { + callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } + }); + return future; + } + + public static CompletableFuture wrap(AsyncCallbacks.ReadEntryCallback callback, Object ctx) { + final var future = new CompletableFuture(); + future.whenComplete((entries, throwable) -> { + if (throwable == null) { + callback.readEntryComplete(entries, ctx); + } else { + callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } + }); + return future; + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 9c6c79eb95b40..00149eadda953 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -115,13 +115,12 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { long startEntry = invocationOnMock.getArgument(1); long endEntry = invocationOnMock.getArgument(2); IntSupplier expectedReadCount = invocationOnMock.getArgument(3); - AsyncCallbacks.ReadEntriesCallback callback = invocationOnMock.getArgument(4); - Object ctx = invocationOnMock.getArgument(5); - pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, callback, ctx); + CompletableFuture> future = invocationOnMock.getArgument(4); + pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, future); return null; } }).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(), - any(), any(), any(), anyBoolean()); + any(), any(), anyBoolean()); lh = mock(ReadHandle.class); ml = mock(ManagedLedgerImpl.class); From 4392827188385433ff775c07e1ff10d544cbc113 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 15:36:34 +0800 Subject: [PATCH 04/16] Fix PendingReadsManagerTest --- .../impl/cache/PendingReadsManagerTest.java | 209 +++++------------- 1 file changed, 59 insertions(+), 150 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java index 00149eadda953..91e2026a3be3a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManagerTest.java @@ -27,8 +27,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertSame; -import static org.testng.Assert.assertTrue; import io.opentelemetry.api.OpenTelemetry; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -45,20 +43,13 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.IntSupplier; -import java.util.stream.Collectors; -import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.lang3.tuple.Pair; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -67,8 +58,6 @@ @Slf4j public class PendingReadsManagerTest { - static final Object CTX = "foo"; - static final Object CTX2 = "far"; static final long LEDGER_ID = 123414L; private final Map, AtomicInteger> entryRangeReadCount = new ConcurrentHashMap<>(); ExecutorService orderedExecutor; @@ -107,20 +96,13 @@ void setupMocks() { mock(ScheduledExecutorService.class), OpenTelemetry.noop()); when(rangeEntryCache.getPendingReadsLimiter()).thenReturn(inflighReadsLimiter); pendingReadsManager = new PendingReadsManager(rangeEntryCache); - doAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) throws Throwable { - log.info("rangeEntryCache asyncReadEntry0 {}", invocationOnMock); - ReadHandle rh = invocationOnMock.getArgument(0); - long startEntry = invocationOnMock.getArgument(1); - long endEntry = invocationOnMock.getArgument(2); - IntSupplier expectedReadCount = invocationOnMock.getArgument(3); - CompletableFuture> future = invocationOnMock.getArgument(4); - pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount, future); - return null; - } - }).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(), - any(), any(), anyBoolean()); + doAnswer(invocationOnMock -> { + log.info("rangeEntryCache asyncReadEntry0 {}", invocationOnMock); + long startEntry = invocationOnMock.getArgument(1); + long endEntry = invocationOnMock.getArgument(2); + IntSupplier expectedReadCount = invocationOnMock.getArgument(3); + return pendingReadsManager.readEntries(lh, startEntry, endEntry, expectedReadCount); + }).when(rangeEntryCache).asyncReadEntry0(any(), anyLong(), anyLong(), any(), anyBoolean()); lh = mock(ReadHandle.class); ml = mock(ManagedLedgerImpl.class); @@ -129,32 +111,6 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { entryRangeReadCount.clear(); } - - @Data - private static class CapturingReadEntriesCallback extends CompletableFuture - implements AsyncCallbacks.ReadEntriesCallback { - List entries; - Object ctx; - Throwable error; - - @Override - public synchronized void readEntriesComplete(List entries, Object ctx) { - this.entries = entries.stream().map(Entry::getPosition).collect(Collectors.toList()); - this.ctx = ctx; - this.error = null; - this.complete(null); - } - - @Override - public synchronized void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - this.entries = null; - this.ctx = ctx; - this.error = exception; - this.completeExceptionally(exception); - } - - } - private static List buildList(long start, long end) { List result = new ArrayList<>(); for (long i = start; i <= end; i++) { @@ -166,7 +122,7 @@ private static List buildList(long start, long end) { } - private void verifyRange(List entries, long firstEntry, long endEntry) { + private void verifyRange(List entries, long firstEntry, long endEntry) { int pos = 0; log.info("verifyRange numEntries {}", entries.size()); for (long entry = firstEntry; entry <= endEntry; entry++) { @@ -224,18 +180,13 @@ public void simpleRead() throws Exception { PreparedReadFromStorage read1 = prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); + final var future = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); // complete the read read1.storageReadCompleted(); // wait for the callback to complete - callback.get(); - assertSame(callback.getCtx(), CTX); - - // verify - verifyRange(callback.entries, firstEntry, endEntry); + verifyRange(future.get(), firstEntry, endEntry); } @@ -250,28 +201,21 @@ public void simpleConcurrentReadPerfectMatch() throws Exception { prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount); PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); - - CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback2, CTX2); + final var future1 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); + final var future2 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); // complete the read from BK // only one read completes 2 callbacks read1.storageReadCompleted(); - callback.get(); - callback2.get(); - - assertSame(callback.getCtx(), CTX); - assertSame(callback2.getCtx(), CTX2); - - verifyRange(callback.entries, firstEntry, endEntry); - verifyRange(callback2.entries, firstEntry, endEntry); + final var entries1 = future1.get(); + final var entries2 = future2.get(); + verifyRange(entries1, firstEntry, endEntry); + verifyRange(entries2, firstEntry, endEntry); int pos = 0; for (long entry = firstEntry; entry <= endEntry; entry++) { - assertTrue(callback.entries.get(pos).compareTo(callback2.entries.get(pos)) == 0); + assertEquals(entries1.get(pos).getPosition().compareTo(entries2.get(pos).getPosition()), 0); pos++; } @@ -292,32 +236,24 @@ public void simpleConcurrentReadIncluding() throws Exception { prepareReadFromStorage(lh, rangeEntryCache, firstEntry, endEntry, expectedReadCount); PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); - - - CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, - CTX2); + final var future1 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); + final var future2 = pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, + expectedReadCount); // complete the read from BK // only one read completes 2 callbacks read1.storageReadCompleted(); - callback.get(); - callback2.get(); - - assertSame(callback.getCtx(), CTX); - assertSame(callback2.getCtx(), CTX2); - - verifyRange(callback.entries, firstEntry, endEntry); - verifyRange(callback2.entries, firstEntrySecondRead, endEntrySecondRead); + final var entries1 = future1.get(); + final var entries2 = future2.get(); + verifyRange(entries1, firstEntry, endEntry); + verifyRange(entries2, firstEntrySecondRead, endEntrySecondRead); int pos = 0; for (long entry = firstEntry; entry <= endEntry; entry++) { if (entry >= firstEntrySecondRead && entry <= endEntrySecondRead) { int posInSecondList = (int) (pos - (firstEntrySecondRead - firstEntry)); - assertTrue(callback.entries.get(pos).compareTo(callback2.entries.get(posInSecondList)) == 0); + assertEquals(entries1.get(pos).getPosition().compareTo(entries2.get(posInSecondList).getPosition()), 0); } pos++; } @@ -342,26 +278,20 @@ public void simpleConcurrentReadMissingLeft() throws Exception { prepareReadFromStorage(lh, rangeEntryCache, firstEntrySecondRead, firstEntry - 1, expectedReadCount); PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); - - CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, - CTX2); + final var future1 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); + final var future2 = pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, + expectedReadCount); // complete the read from BK read1.storageReadCompleted(); // the first read can move forward - callback.get(); + final var entries1 = future1.get(); readForLeft.storageReadCompleted(); - callback2.get(); - - assertSame(callback.getCtx(), CTX); - assertSame(callback2.getCtx(), CTX2); + final var entries2 = future2.get(); - verifyRange(callback.entries, firstEntry, endEntry); - verifyRange(callback2.entries, firstEntrySecondRead, endEntrySecondRead); + verifyRange(entries1, firstEntry, endEntry); + verifyRange(entries2, firstEntrySecondRead, endEntrySecondRead); } @@ -383,26 +313,20 @@ public void simpleConcurrentReadMissingRight() throws Exception { prepareReadFromStorage(lh, rangeEntryCache, endEntry + 1, endEntrySecondRead, expectedReadCount); PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); - - CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, - CTX2); + final var future1 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); + final var future2 = pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, + expectedReadCount); // complete the read from BK read1.storageReadCompleted(); // the first read can move forward - callback.get(); + final var entries1 = future1.get(); readForRight.storageReadCompleted(); - callback2.get(); - - assertSame(callback.getCtx(), CTX); - assertSame(callback2.getCtx(), CTX2); + final var entries2 = future2.get(); - verifyRange(callback.entries, firstEntry, endEntry); - verifyRange(callback2.entries, firstEntrySecondRead, endEntrySecondRead); + verifyRange(entries1, firstEntry, endEntry); + verifyRange(entries2, firstEntrySecondRead, endEntrySecondRead); } @@ -427,27 +351,21 @@ public void simpleConcurrentReadMissingBoth() throws Exception { prepareReadFromStorage(lh, rangeEntryCache, endEntry + 1, endEntrySecondRead, expectedReadCount); PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); - - CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, - CTX2); + final var future1 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); + final var future2 = pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, + expectedReadCount); // complete the read from BK read1.storageReadCompleted(); // the first read can move forward - callback.get(); + final var entries1 = future1.get(); readForLeft.storageReadCompleted(); readForRight.storageReadCompleted(); - callback2.get(); + final var entries2 = future2.get(); - assertSame(callback.getCtx(), CTX); - assertSame(callback2.getCtx(), CTX2); - - verifyRange(callback.entries, firstEntry, endEntry); - verifyRange(callback2.entries, firstEntrySecondRead, endEntrySecondRead); + verifyRange(entries1, firstEntry, endEntry); + verifyRange(entries2, firstEntrySecondRead, endEntrySecondRead); } @@ -470,35 +388,26 @@ public void simpleConcurrentReadNoMatch() throws Exception { expectedReadCount); PendingReadsManager pendingReadsManager = new PendingReadsManager(rangeEntryCache); - CapturingReadEntriesCallback callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount, callback, CTX); - - CapturingReadEntriesCallback callback2 = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, expectedReadCount, callback2, - CTX2); + final var future1 = pendingReadsManager.readEntries(lh, firstEntry, endEntry, expectedReadCount); + final var future2 = pendingReadsManager.readEntries(lh, firstEntrySecondRead, endEntrySecondRead, + expectedReadCount); read1.storageReadCompleted(); - callback.get(); + final var entries1 = future1.get(); read2.storageReadCompleted(); - callback2.get(); - - assertSame(callback.getCtx(), CTX); - assertSame(callback2.getCtx(), CTX2); + final var entries2 = future2.get(); - verifyRange(callback.entries, firstEntry, endEntry); - verifyRange(callback2.entries, firstEntrySecondRead, endEntrySecondRead); + verifyRange(entries1, firstEntry, endEntry); + verifyRange(entries2, firstEntrySecondRead, endEntrySecondRead); } @Test public void concurrentReadOnOverlappedEntryRanges() throws Exception { - final var readFutures = new ArrayList(); - final BiConsumer readEntries = (firstEntry, lastEntry) -> { - final var callback = new CapturingReadEntriesCallback(); - pendingReadsManager.readEntries(lh, firstEntry, lastEntry, () -> 0, callback, CTX); - readFutures.add(callback); - }; + final var readFutures = new ArrayList>>(); + final BiConsumer readEntries = (firstEntry, lastEntry) -> + readFutures.add(pendingReadsManager.readEntries(lh, firstEntry, lastEntry, () -> 0)); final BiFunction mockReadFromStorage = (firstEntry, lastEntry) -> prepareReadFromStorage(lh, rangeEntryCache, firstEntry, lastEntry, () -> 0); @@ -511,15 +420,15 @@ public void concurrentReadOnOverlappedEntryRanges() throws Exception { read1.storageReadCompleted(); readFutures.get(1).get(1, TimeUnit.SECONDS); - assertEquals(readFutures.get(1).getEntries().size(), 21); + assertEquals(readFutures.get(1).get().size(), 21); read0.storageReadCompleted(); readFutures.get(0).get(1, TimeUnit.SECONDS); - assertEquals(readFutures.get(0).getEntries().size(), 61); + assertEquals(readFutures.get(0).get().size(), 61); read2.storageReadCompleted(); readFutures.get(2).get(1, TimeUnit.SECONDS); - assertEquals(readFutures.get(2).getEntries().size(), 91); + assertEquals(readFutures.get(2).get().size(), 91); log.info("entryRangeReadCount: {}", entryRangeReadCount); final var keys = Set.of(Pair.of(10L, 70L), Pair.of(71L, 79L), From 0df5aae828418f7f0729943ab0c80c37bc4e3151 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 16:39:24 +0800 Subject: [PATCH 05/16] Make build pass (TODO: fix RangeEntryCacheImplTest and EntryCacheManagerTest) --- .../mledger/impl/ManagedLedgerImpl.java | 13 +++-- .../mledger/util/CallbackUtils.java | 44 ++++++--------- .../mledger/impl/EntryCacheManagerTest.java | 20 +------ .../mledger/impl/EntryCacheTest.java | 15 +----- .../InflightReadsLimiterIntegrationTest.java | 54 +++++++------------ .../impl/cache/RangeEntryCacheImplTest.java | 52 +++++++----------- 6 files changed, 66 insertions(+), 132 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index d157677d2105e..51f0cde38efbd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -133,6 +133,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext; import org.apache.bookkeeper.mledger.util.CallbackMutex; +import org.apache.bookkeeper.mledger.util.CallbackUtils; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; import org.apache.bookkeeper.net.BookieId; @@ -2407,9 +2408,11 @@ protected void asyncReadEntry(ReadHandle ledger, Position position, ReadEntryCal ReadEntryCallbackWrapper readCallback = ReadEntryCallbackWrapper.create(name, position.getLedgerId(), position.getEntryId(), callback, readOpCount, createdTime, ctx); lastReadCallback = readCallback; - entryCache.asyncReadEntry(ledger, position, readCallback, readOpCount); + entryCache.asyncReadEntry(ledger, position).whenComplete((entry, throwable) -> + CallbackUtils.complete(entry, throwable, readCallback, readOpCount)); } else { - entryCache.asyncReadEntry(ledger, position, callback, ctx); + entryCache.asyncReadEntry(ledger, position).whenComplete((entry, throwable) -> + CallbackUtils.complete(entry, throwable, callback, ctx)); } } @@ -2423,9 +2426,11 @@ protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry ReadEntryCallbackWrapper readCallback = ReadEntryCallbackWrapper.create(name, ledger.getId(), firstEntry, opReadEntry, readOpCount, createdTime, ctx); lastReadCallback = readCallback; - entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount, readCallback, readOpCount); + entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount).whenComplete( + (entries, throwable) -> CallbackUtils.complete(entries, throwable, readCallback, readOpCount)); } else { - entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount, opReadEntry, ctx); + entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount).whenComplete( + (entries, throwable) -> CallbackUtils.complete(entries, throwable, opReadEntry, ctx)); } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java index da10cb9fae1e6..760cb10f905f1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java @@ -19,42 +19,28 @@ package org.apache.bookkeeper.mledger.util; import java.util.List; -import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.jspecify.annotations.Nullable; -/** - * Currently, the managed ledger APIs mostly accept a callback and a context that is passed to the callback, if the - * callback is called twice unexpectedly, the caller side might fail, e.g. the context is a recyclable object whose - * `recycle()` method is called when it's passed to the callback. - * This util class delegates the callback and the corresponding context object to a {@link CompletableFuture}, which - * guarantees the callback can be called exactly once in {@link CompletableFuture#whenComplete} when the future is - * completed. - */ public class CallbackUtils { - public static CompletableFuture> wrap(AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { - final var future = new CompletableFuture>(); - future.whenComplete((entries, throwable) -> { - if (throwable == null) { - callback.readEntriesComplete(entries, ctx); - } else { - callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); - } - }); - return future; + public static void complete(List entries, @Nullable Throwable throwable, + AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { + if (throwable == null) { + callback.readEntriesComplete(entries, ctx); + } else { + callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } } - public static CompletableFuture wrap(AsyncCallbacks.ReadEntryCallback callback, Object ctx) { - final var future = new CompletableFuture(); - future.whenComplete((entries, throwable) -> { - if (throwable == null) { - callback.readEntryComplete(entries, ctx); - } else { - callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); - } - }); - return future; + public static void complete(Entry entry, @Nullable Throwable throwable, AsyncCallbacks.ReadEntryCallback callback, + Object ctx) { + if (throwable == null) { + callback.readEntryComplete(entry, ctx); + } else { + callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index 49743d9386c90..b019b483c61fb 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -29,15 +29,12 @@ import java.util.List; import java.util.Optional; import java.util.Random; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; @@ -47,7 +44,6 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.awaitility.Awaitility; -import org.testng.Assert; import org.testng.annotations.Test; public class EntryCacheManagerTest extends MockedBookKeeperTestCase { @@ -408,23 +404,11 @@ void entryCacheDisabledAsyncReadEntry() throws Exception { EntryCacheManager cacheManager = factory.getEntryCacheManager(); EntryCache entryCache = cacheManager.getEntryCache(ml1); - final CountDownLatch counter = new CountDownLatch(1); when(ml1.getLastConfirmedEntry()).thenReturn(PositionFactory.create(1L, 1L)); when(ml1.getOptionalLedgerInfo(lh.getId())).thenReturn(Optional.of(mock( MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); - entryCache.asyncReadEntry(lh, PositionFactory.create(1L, 1L), new AsyncCallbacks.ReadEntryCallback() { - public void readEntryComplete(Entry entry, Object ctx) { - Assert.assertNotEquals(entry, null); - entry.release(); - counter.countDown(); - } - - public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - counter.countDown(); - } - }, null); - counter.await(); + final var entry = entryCache.asyncReadEntry(lh, PositionFactory.create(1L, 1L)).get(); + entry.release(); verify(lh).readUnconfirmedAsync(anyLong(), anyLong()); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java index 0fe889a2da5ec..255fcd105458a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java @@ -41,7 +41,6 @@ import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; @@ -255,19 +254,7 @@ static ReadHandle getLedgerHandle() { private List readEntry(EntryCache entryCache, ReadHandle lh, long firstEntry, long lastEntry, IntSupplier expectedReadCount, Consumer assertion) throws InterruptedException { - final var future = new CompletableFuture>(); - entryCache.asyncReadEntry(lh, firstEntry, lastEntry, expectedReadCount, - new ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx) { - future.complete(entries); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(exception); - } - }, null); + final var future = entryCache.asyncReadEntry(lh, firstEntry, lastEntry, expectedReadCount); try { final var entries = future.get(); assertNull(assertion); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java index 01a8808a2b96c..d765da86e7f7c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -21,7 +21,6 @@ import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.doAnswer; import io.netty.util.concurrent.DefaultThreadFactory; -import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.UUID; @@ -30,10 +29,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.LedgerEntries; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.impl.cache.InflightReadsLimiter; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl; @@ -137,10 +134,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { doAnswer(answer).when(spyCurrentLedger).readUnconfirmedAsync(anyLong(), anyLong()); // Initialize "entryCache.estimatedEntrySize" to the correct value. - Object ctx = new Object(); - SimpleReadEntriesCallback cb0 = new SimpleReadEntriesCallback(); - entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, () -> 1, cb0, ctx); - cb0.entries.join(); + entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, () -> 1).get(); int sizePerEntry = Long.valueOf(entryCache.getEstimatedEntrySize(ml.currentLedger)).intValue(); Awaitility.await().untilAsserted(() -> { long remainingBytes = limiter.getRemainingBytes(); @@ -149,19 +143,30 @@ public void testPreciseLimitation(String missingCase) throws Exception { log.info("remainingBytes 0: {}", limiter.getRemainingBytes()); // Concurrency reading. - - SimpleReadEntriesCallback cb1 = new SimpleReadEntriesCallback(); - SimpleReadEntriesCallback cb2 = new SimpleReadEntriesCallback(); + final var future1 = new CompletableFuture>(); threadFactory.newThread(() -> { - entryCache.asyncReadEntry(spyCurrentLedger, start1, end1, () -> 1, cb1, ctx); + entryCache.asyncReadEntry(spyCurrentLedger, start1, end1, () -> 1).whenComplete((entries, throwable) -> { + if (throwable == null) { + future1.complete(entries); + } else { + future1.completeExceptionally(throwable); + } + }); }).start(); + final var future2 = new CompletableFuture>(); threadFactory.newThread(() -> { try { firstReadingStarted.await(); } catch (InterruptedException e) { throw new RuntimeException(e); } - entryCache.asyncReadEntry(spyCurrentLedger, start2, end2, () -> 1, cb2, ctx); + entryCache.asyncReadEntry(spyCurrentLedger, start2, end2, () -> 1).whenComplete((entries, throwable) -> { + if (throwable == null) { + future2.complete(entries); + } else { + future2.completeExceptionally(throwable); + } + }); }).start(); long bytesAcquired1 = calculateBytesSizeBeforeFirstReading(readCount1 + readCount2, sizePerEntry); @@ -176,7 +181,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { // Complete the read1. Thread.sleep(3000); readCompleteSignal1.countDown(); - cb1.entries.join(); + future1.get(); long bytesAcquired2 = calculateBytesSizeBeforeFirstReading(readCount2, sizePerEntry); long remainingBytesExpected2 = totalCapacity - bytesAcquired2; log.info("acquired : {}", bytesAcquired2); @@ -187,7 +192,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { }); readCompleteSignal2.countDown(); - cb2.entries.join(); + future2.get(); Awaitility.await().untilAsserted(() -> { long remainingBytes = limiter.getRemainingBytes(); log.info("remainingBytes 2: {}", remainingBytes); @@ -201,25 +206,4 @@ public void testPreciseLimitation(String missingCase) throws Exception { private long calculateBytesSizeBeforeFirstReading(int entriesCount, int perEntrySize) { return entriesCount * perEntrySize; } - - class SimpleReadEntriesCallback implements AsyncCallbacks.ReadEntriesCallback { - - CompletableFuture> entries = new CompletableFuture<>(); - - @Override - public void readEntriesComplete(List entriesRead, Object ctx) { - List list = new ArrayList<>(entriesRead.size()); - for (Entry entry : entriesRead) { - byte b = entry.getDataBuffer().readByte(); - list.add(b); - entry.release(); - } - this.entries.complete(list); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - this.entries.completeExceptionally(exception); - } - } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java index 2922a3d267982..72416da40096a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java @@ -31,7 +31,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.function.IntSupplier; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.AsyncCallbacks; @@ -81,7 +80,7 @@ public void setup() { } callback.readEntriesComplete(entries, ctx); return null; - }).when(pendingReadsManager).readEntries(any(), anyLong(), anyLong(), any(), any(), any()); + }).when(pendingReadsManager).readEntries(any(), anyLong(), anyLong(), any()); rangeEntryCache = new RangeEntryCacheImpl(mockEntryCacheManager, mockManagedLedger, false, mockRangeCacheRemovalQueue, EntryLengthFunction.DEFAULT, pendingReadsManager); @@ -95,8 +94,8 @@ public void testPartialCachingWithMiddleEntryInCache() { Entry entry = EntryImpl.create(1, 50, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(51L), eq(99L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(51L), eq(99L), any()); } @Test @@ -104,7 +103,7 @@ public void testPartialCachingWithFirstEntryInCache() { Entry entry = EntryImpl.create(1, 0, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(1L), eq(99L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(1L), eq(99L), any()); } @Test @@ -112,7 +111,7 @@ public void testPartialCachingWithLastEntryInCache() { Entry entry = EntryImpl.create(1, 99, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(98L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(98L), any()); } @Test @@ -122,8 +121,8 @@ public void testPartialCachingWithMiddleRangeInCache() { entry = EntryImpl.create(1, 51, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(52L), eq(99L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(52L), eq(99L), any()); } @Test @@ -133,7 +132,7 @@ public void testPartialCachingWithFirstRangeInCache() { entry = EntryImpl.create(1, 1, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(2L), eq(99L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(2L), eq(99L), any()); } @Test @@ -143,7 +142,7 @@ public void testPartialCachingWithLastRangeInCache() { EntryImpl.create(1, 99, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(97L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(97L), any()); } @Test @@ -159,11 +158,11 @@ public void testPartialCachingWithMultipleEntriesInCache() { entry = EntryImpl.create(1, 78, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(4L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(6L), eq(14L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(16L), eq(74L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(77L), eq(77L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(79L), eq(99L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(4L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(6L), eq(14L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(16L), eq(74L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(77L), eq(77L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(79L), eq(99L), any()); } @Test @@ -176,26 +175,15 @@ public void testPartialCachingWithMultipleEntriesInCacheWhilePartialReadFails() System.out.println("Injecting test failure for readEntries"); callback.readEntriesFailed(new ManagedLedgerException("Injected test failure"), ctx); return null; - }).when(pendingReadsManager).readEntries(any(), eq(51L), eq(99L), any(), any(), any()); + }).when(pendingReadsManager).readEntries(any(), eq(51L), eq(99L), any()); performReadAndValidateResult(); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(51L), eq(99L), any(), any(), any()); - verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(99L), any(), any(), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(51L), eq(99L), any()); + verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(99L), any()); } private void performReadAndValidateResult() { - CompletableFuture> future = new CompletableFuture<>(); - rangeEntryCache.asyncReadEntry(lh, 0, 99, expectedReadCount, new AsyncCallbacks.ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx) { - future.complete(entries); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - future.completeExceptionally(exception); - } - }, null); + final var future = rangeEntryCache.asyncReadEntry(lh, 0, 99, expectedReadCount); assertThat(future).isCompleted().satisfies(f -> { List entries = f.getNow(null); assertThat(entries).hasSize(100); @@ -206,4 +194,4 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { } }); } -} \ No newline at end of file +} From e639672a0357eca11868abf19acb2140c73f9dbf Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 16:48:31 +0800 Subject: [PATCH 06/16] Fix RangeEntryCacheImplTest --- .../mledger/impl/cache/RangeEntryCacheImplTest.java | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java index 72416da40096a..3f23d0f95e465 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImplTest.java @@ -31,9 +31,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.function.IntSupplier; import org.apache.bookkeeper.client.api.ReadHandle; -import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.EntryImpl; @@ -72,14 +72,11 @@ public void setup() { doAnswer(invocation -> { long firstEntry = invocation.getArgument(1); long lastEntry = invocation.getArgument(2); - AsyncCallbacks.ReadEntriesCallback callback = invocation.getArgument(4); - Object ctx = invocation.getArgument(5); List entries = new ArrayList<>((int) (lastEntry - firstEntry + 1)); for (long entryId = firstEntry; entryId <= lastEntry; entryId++) { entries.add(EntryImpl.create(1, entryId, Unpooled.EMPTY_BUFFER)); } - callback.readEntriesComplete(entries, ctx); - return null; + return CompletableFuture.completedFuture(entries); }).when(pendingReadsManager).readEntries(any(), anyLong(), anyLong(), any()); rangeEntryCache = new RangeEntryCacheImpl(mockEntryCacheManager, mockManagedLedger, false, mockRangeCacheRemovalQueue, @@ -170,11 +167,8 @@ public void testPartialCachingWithMultipleEntriesInCacheWhilePartialReadFails() Entry entry = EntryImpl.create(1, 50, Unpooled.EMPTY_BUFFER); rangeEntryCache.insert(entry); doAnswer(invocation -> { - AsyncCallbacks.ReadEntriesCallback callback = invocation.getArgument(4); - Object ctx = invocation.getArgument(5); System.out.println("Injecting test failure for readEntries"); - callback.readEntriesFailed(new ManagedLedgerException("Injected test failure"), ctx); - return null; + return CompletableFuture.failedFuture(new ManagedLedgerException("Injected test failure")); }).when(pendingReadsManager).readEntries(any(), eq(51L), eq(99L), any()); performReadAndValidateResult(); verify(pendingReadsManager, times(1)).readEntries(any(), eq(0L), eq(49L), any()); From f98f31e62869f1b17fe1ea50acd585415891b36f Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 17:24:19 +0800 Subject: [PATCH 07/16] Fix InflightReadsLimiterIntegrationTest --- .../impl/cache/RangeEntryCacheImpl.java | 32 ++++++++----------- .../InflightReadsLimiterIntegrationTest.java | 13 ++++++-- 2 files changed, 24 insertions(+), 21 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java index 887ccf310f438..b07f55575c3a6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/RangeEntryCacheImpl.java @@ -347,27 +347,23 @@ CompletableFuture> doAsyncReadEntriesWithAcquiredPermits( return CompletableFuture.failedFuture(new ManagedLedgerException.TooManyRequestsException(message)); } InflightReadsLimiter pendingReadsLimiter = getPendingReadsLimiter(); - final var future = doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, - expectedReadCount); - future.thenAccept(entries -> { - if (!entries.isEmpty()) { - // release permits only when entries have been handled - AtomicInteger remainingCount = new AtomicInteger(entries.size()); - for (Entry entry : entries) { - ((EntryImpl) entry).onDeallocate(() -> { - if (remainingCount.decrementAndGet() <= 0) { - pendingReadsLimiter.release(handle); - } - }); - } - } else { + return doAsyncReadEntriesByPosition(lh, firstPosition, lastPosition, numberOfEntries, + expectedReadCount + ).whenComplete((entries, e) -> { + if (e != null || entries.isEmpty()) { pendingReadsLimiter.release(handle); + return; + } + // release permits only when entries have been handled + AtomicInteger remainingCount = new AtomicInteger(entries.size()); + for (Entry entry : entries) { + ((EntryImpl) entry).onDeallocate(() -> { + if (remainingCount.decrementAndGet() <= 0) { + pendingReadsLimiter.release(handle); + } + }); } - }).exceptionally(e -> { - pendingReadsLimiter.release(handle); - return null; }); - return future; } CompletableFuture> doAsyncReadEntriesByPosition( diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java index d765da86e7f7c..63c3c668aba64 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/InflightReadsLimiterIntegrationTest.java @@ -134,7 +134,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { doAnswer(answer).when(spyCurrentLedger).readUnconfirmedAsync(anyLong(), anyLong()); // Initialize "entryCache.estimatedEntrySize" to the correct value. - entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, () -> 1).get(); + releaseEntries(entryCache.asyncReadEntry(spyCurrentLedger, 125, 125, () -> 1).get()); int sizePerEntry = Long.valueOf(entryCache.getEstimatedEntrySize(ml.currentLedger)).intValue(); Awaitility.await().untilAsserted(() -> { long remainingBytes = limiter.getRemainingBytes(); @@ -181,7 +181,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { // Complete the read1. Thread.sleep(3000); readCompleteSignal1.countDown(); - future1.get(); + releaseEntries(future1.get()); long bytesAcquired2 = calculateBytesSizeBeforeFirstReading(readCount2, sizePerEntry); long remainingBytesExpected2 = totalCapacity - bytesAcquired2; log.info("acquired : {}", bytesAcquired2); @@ -192,7 +192,7 @@ public void testPreciseLimitation(String missingCase) throws Exception { }); readCompleteSignal2.countDown(); - future2.get(); + releaseEntries(future2.get()); Awaitility.await().untilAsserted(() -> { long remainingBytes = limiter.getRemainingBytes(); log.info("remainingBytes 2: {}", remainingBytes); @@ -206,4 +206,11 @@ public void testPreciseLimitation(String missingCase) throws Exception { private long calculateBytesSizeBeforeFirstReading(int entriesCount, int perEntrySize) { return entriesCount * perEntrySize; } + + // The permits will only be released after entries are released + private void releaseEntries(List entries) { + for (Entry entry : entries) { + entry.release(); + } + } } From 460140fc414aefa3814d3b363ac2fe3ccb4f3f53 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 18:10:24 +0800 Subject: [PATCH 08/16] Never expose OpReadEntry.ctx --- .../mledger/impl/ManagedCursorImpl.java | 4 +- .../mledger/impl/ManagedLedgerImpl.java | 221 ++++-------------- .../bookkeeper/mledger/impl/OpReadEntry.java | 30 ++- .../mledger/impl/ManagedCursorTest.java | 4 +- 4 files changed, 63 insertions(+), 196 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f74e834e8be1e..b5421988accd3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -2962,7 +2962,7 @@ protected void closeWaitingCursor() { OpReadEntry opReadEntry = WAITING_READ_OP_UPDATER.getAndSet(this, OpReadEntry.WAITING_READ_OP_FOR_CLOSED_CURSOR); if (opReadEntry != null && opReadEntry != OpReadEntry.WAITING_READ_OP_FOR_CLOSED_CURSOR) { - opReadEntry.readEntriesFailed(new CursorAlreadyClosedException("Cursor is closing"), opReadEntry.ctx); + opReadEntry.readEntriesFailed(new CursorAlreadyClosedException("Cursor is closing")); } } @@ -3532,7 +3532,7 @@ void notifyEntriesAvailable() { log.debug("[{}] [{}] Cursor is already closed, ignoring notification", ledger.getName(), name); } opReadEntry.readEntriesFailed(new ManagedLedgerException.CursorAlreadyClosedException( - "Cursor was already closed"), opReadEntry.ctx); + "Cursor was already closed")); return; } PENDING_READ_OPS_UPDATER.incrementAndGet(this); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 51f0cde38efbd..5a54504c85ae8 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -29,8 +29,6 @@ import com.google.common.collect.Range; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import io.netty.util.Recycler; -import io.netty.util.Recycler.Handle; import java.time.Clock; import java.util.ArrayList; import java.util.Collection; @@ -92,7 +90,6 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OffloadCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback; -import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.TerminateCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.UpdatePropertiesCallback; @@ -330,11 +327,10 @@ public boolean isFenced() { .newUpdater(ManagedLedgerImpl.class, "addOpCount"); private volatile long addOpCount = 0; - // last read-operation's callback to check read-timeout on it. - private volatile ReadEntryCallbackWrapper lastReadCallback = null; - private static final AtomicReferenceFieldUpdater - LAST_READ_CALLBACK_UPDATER = AtomicReferenceFieldUpdater - .newUpdater(ManagedLedgerImpl.class, ReadEntryCallbackWrapper.class, "lastReadCallback"); + private volatile PendingReadEntriesOp lastReadEntriesOp = null; + private static final AtomicReferenceFieldUpdater + LAST_READ_ENTRIES_OP = AtomicReferenceFieldUpdater + .newUpdater(ManagedLedgerImpl.class, PendingReadEntriesOp.class, "lastReadEntriesOp"); /** * Queue of pending entries to be added to the managed ledger. Typically, entries are queued when a new ledger is. @@ -2065,7 +2061,7 @@ void clearPendingAddEntries(ManagedLedgerException e) { void asyncReadEntries(OpReadEntry opReadEntry) { final State state = STATE_UPDATER.get(this); if (state.isFenced() || state == State.Closed) { - opReadEntry.readEntriesFailed(new ManagedLedgerFencedException(), opReadEntry.ctx); + opReadEntry.readEntriesFailed(new ManagedLedgerFencedException()); return; } @@ -2106,8 +2102,7 @@ && isLedgerFullyAcked(ledgerId, ledgerInfo, opReadEntry.cursor)) { -> { log.error("[{}] Error opening ledger for reading at position {} - {}", name, opReadEntry.readPosition, ex.getMessage()); - opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(ex.getCause()), - opReadEntry.ctx); + opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(ex.getCause())); return null; }); } @@ -2396,183 +2391,58 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) log.debug("[{}] Reading entries from ledger {} - first={} last={}", name, ledger.getId(), firstEntry, lastEntry); } - asyncReadEntry(ledger, firstEntry, lastEntry, opReadEntry, opReadEntry.ctx); + asyncReadEntry(ledger, firstEntry, lastEntry, opReadEntry); } protected void asyncReadEntry(ReadHandle ledger, Position position, ReadEntryCallback callback, Object ctx) { mbean.addEntriesRead(1); if (config.getReadEntryTimeoutSeconds() > 0) { - // set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); long createdTime = System.nanoTime(); - ReadEntryCallbackWrapper readCallback = ReadEntryCallbackWrapper.create(name, position.getLedgerId(), - position.getEntryId(), callback, readOpCount, createdTime, ctx); - lastReadCallback = readCallback; - entryCache.asyncReadEntry(ledger, position).whenComplete((entry, throwable) -> - CallbackUtils.complete(entry, throwable, readCallback, readOpCount)); + final var future = entryCache.asyncReadEntry(ledger, position).thenApply(List::of); + future.whenComplete((entries, throwable) -> { + if (throwable == null) { + callback.readEntryComplete(entries.get(0), ctx); + } else { + callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } + }); + lastReadEntriesOp = new PendingReadEntriesOp(position.getLedgerId(), position.getEntryId(), createdTime, + readOpCount, future); } else { entryCache.asyncReadEntry(ledger, position).whenComplete((entry, throwable) -> CallbackUtils.complete(entry, throwable, callback, ctx)); } } - protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry, OpReadEntry opReadEntry, - Object ctx) { + protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry, OpReadEntry opReadEntry) { IntSupplier expectedReadCount = opReadEntry.cursor::getNumberOfCursorsAtSamePositionOrBefore; if (config.getReadEntryTimeoutSeconds() > 0) { - // set readOpCount to uniquely validate if ReadEntryCallbackWrapper is already recycled long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); long createdTime = System.nanoTime(); - ReadEntryCallbackWrapper readCallback = ReadEntryCallbackWrapper.create(name, ledger.getId(), firstEntry, - opReadEntry, readOpCount, createdTime, ctx); - lastReadCallback = readCallback; - entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount).whenComplete( - (entries, throwable) -> CallbackUtils.complete(entries, throwable, readCallback, readOpCount)); + final var future = entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount); + future.whenComplete((entries, throwable) -> { + if (throwable == null) { + opReadEntry.readEntriesComplete(entries); + } else { + opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable)); + } + }); + lastReadEntriesOp = new PendingReadEntriesOp(ledger.getId(), firstEntry, createdTime, readOpCount, future); } else { entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount).whenComplete( - (entries, throwable) -> CallbackUtils.complete(entries, throwable, opReadEntry, ctx)); - } - } - - static final class ReadEntryCallbackWrapper implements ReadEntryCallback, ReadEntriesCallback { - - volatile ReadEntryCallback readEntryCallback; - volatile ReadEntriesCallback readEntriesCallback; - String name; - long ledgerId; - long entryId; - volatile long readOpCount = -1; - private static final AtomicLongFieldUpdater READ_OP_COUNT_UPDATER = - AtomicLongFieldUpdater.newUpdater(ReadEntryCallbackWrapper.class, "readOpCount"); - volatile long createdTime = -1; - volatile Object cntx; - - final Handle recyclerHandle; - - private ReadEntryCallbackWrapper(Handle recyclerHandle) { - this.recyclerHandle = recyclerHandle; - } - - static ReadEntryCallbackWrapper create(String name, long ledgerId, long entryId, ReadEntryCallback callback, - long readOpCount, long createdTime, Object ctx) { - ReadEntryCallbackWrapper readCallback = RECYCLER.get(); - readCallback.name = name; - readCallback.ledgerId = ledgerId; - readCallback.entryId = entryId; - readCallback.readEntryCallback = callback; - readCallback.cntx = ctx; - readCallback.readOpCount = readOpCount; - readCallback.createdTime = createdTime; - return readCallback; - } - - static ReadEntryCallbackWrapper create(String name, long ledgerId, long entryId, ReadEntriesCallback callback, - long readOpCount, long createdTime, Object ctx) { - ReadEntryCallbackWrapper readCallback = RECYCLER.get(); - readCallback.name = name; - readCallback.ledgerId = ledgerId; - readCallback.entryId = entryId; - readCallback.readEntriesCallback = callback; - readCallback.cntx = ctx; - readCallback.readOpCount = readOpCount; - readCallback.createdTime = createdTime; - return readCallback; - } - - @Override - public void readEntryComplete(Entry entry, Object ctx) { - long reOpCount = reOpCount(ctx); - ReadEntryCallback callback = this.readEntryCallback; - Object cbCtx = this.cntx; - if (recycle(reOpCount)) { - callback.readEntryComplete(entry, cbCtx); - } else { - if (log.isDebugEnabled()) { - log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); - } - entry.release(); - } - } - - @Override - public void readEntryFailed(ManagedLedgerException exception, Object ctx) { - long reOpCount = reOpCount(ctx); - ReadEntryCallback callback = this.readEntryCallback; - Object cbCtx = this.cntx; - if (recycle(reOpCount)) { - callback.readEntryFailed(exception, cbCtx); - } else { - if (log.isDebugEnabled()) { - log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); - } - } - } - - @Override - public void readEntriesComplete(List returnedEntries, Object ctx) { - long reOpCount = reOpCount(ctx); - ReadEntriesCallback callback = this.readEntriesCallback; - Object cbCtx = this.cntx; - if (recycle(reOpCount)) { - callback.readEntriesComplete(returnedEntries, cbCtx); - } else { - if (log.isDebugEnabled()) { - log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); - } - returnedEntries.forEach(Entry::release); - } - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - long reOpCount = reOpCount(ctx); - ReadEntriesCallback callback = this.readEntriesCallback; - Object cbCtx = this.cntx; - if (recycle(reOpCount)) { - callback.readEntriesFailed(exception, cbCtx); - } else { - if (log.isDebugEnabled()) { - log.debug("[{}] read entry already completed for {}-{}", name, ledgerId, entryId); - } - } - } - - private long reOpCount(Object ctx) { - return (ctx instanceof Long) ? (long) ctx : -1; - } - - public void readFailed(ManagedLedgerException exception, Object ctx) { - if (readEntryCallback != null) { - readEntryFailed(exception, ctx); - } else if (readEntriesCallback != null) { - readEntriesFailed(exception, ctx); - } - // It happens when timeout-thread and read-callback both recycles at the same time. - // this read-callback has already been recycled so, do nothing.. - } - - private boolean recycle(long readOpCount) { - if (readOpCount != -1 - && READ_OP_COUNT_UPDATER.compareAndSet(ReadEntryCallbackWrapper.this, readOpCount, -1)) { - createdTime = -1; - readEntryCallback = null; - readEntriesCallback = null; - ledgerId = -1; - entryId = -1; - name = null; - recyclerHandle.recycle(this); - return true; - } - return false; + (entries, throwable) -> { + if (throwable == null) { + opReadEntry.readEntriesComplete(entries); + } else { + opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable)); + } + }); } + } - private static final Recycler RECYCLER = new Recycler() { - @Override - protected ReadEntryCallbackWrapper newObject(Handle handle) { - return new ReadEntryCallbackWrapper(handle); - } - }; - + record PendingReadEntriesOp(long ledgerId, long entryId, long createdTime, long readOpCount, + CompletableFuture> future) { } @Override @@ -4554,15 +4424,14 @@ private void checkReadTimeout() { if (timeoutSec < 1) { return; } - ReadEntryCallbackWrapper callback = this.lastReadCallback; - long readOpCount = callback != null ? callback.readOpCount : 0; - boolean timeout = callback != null && (TimeUnit.NANOSECONDS - .toSeconds(System.nanoTime() - callback.createdTime) >= timeoutSec); - if (readOpCount > 0 && timeout) { - log.warn("[{}]-{}-{} read entry timeout after {} sec", this.name, this.lastReadCallback.ledgerId, - this.lastReadCallback.entryId, timeoutSec); - callback.readFailed(createManagedLedgerException(BKException.Code.TimeoutException), readOpCount); - LAST_READ_CALLBACK_UPDATER.compareAndSet(this, callback, null); + final var lastReadEntriesOp = this.lastReadEntriesOp; + if (lastReadEntriesOp != null && lastReadEntriesOp.readOpCount > 0 && TimeUnit.NANOSECONDS.toSeconds( + System.nanoTime() - lastReadEntriesOp.createdTime) >= timeoutSec) { + log.warn("[{}]-{}-{} read entry timeout after {} sec", this.name, lastReadEntriesOp.ledgerId, + lastReadEntriesOp.entryId, timeoutSec); + lastReadEntriesOp.future.completeExceptionally(createManagedLedgerException( + BKException.Code.TimeoutException)); + LAST_READ_ENTRIES_OP.compareAndSet(this, lastReadEntriesOp, null); } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index b618a25aa3d75..df8edc058b256 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -34,7 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class OpReadEntry implements ReadEntriesCallback { +class OpReadEntry { static final OpReadEntry WAITING_READ_OP_FOR_CLOSED_CURSOR = new OpReadEntry(); private static final AtomicInteger opReadIdGenerator = new AtomicInteger(1); /** @@ -47,7 +47,7 @@ class OpReadEntry implements ReadEntriesCallback { Position readPosition; private int count; private ReadEntriesCallback callback; - Object ctx; + private Object ctx; // Results private List entries; @@ -110,32 +110,30 @@ private void internalReadEntriesComplete(List returnedEntries) { checkReadCompletion(); } - @Override - public void readEntriesComplete(List returnedEntries, Object ctx) { + public void readEntriesComplete(List returnedEntries) { try { internalReadEntriesComplete(returnedEntries); } catch (Throwable throwable) { log.error("[{}] Fallback to readEntriesFailed for exception in readEntriesComplete", this, throwable); - readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable)); } } - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + public void readEntriesFailed(ManagedLedgerException exception) { try { - internalReadEntriesFailed(exception, ctx); + internalReadEntriesFailed(exception); } catch (Throwable throwable) { // At least we should complete the callback - fail(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + fail(ManagedLedgerException.getManagedLedgerException(throwable)); } } - private void internalReadEntriesFailed(ManagedLedgerException exception, Object ctx) { + private void internalReadEntriesFailed(ManagedLedgerException exception) { cursor.readOperationCompleted(); if (!entries.isEmpty()) { // There were already some entries that were read before, we can return them - complete(ctx); + complete(); } else if (!cursor.isClosed() && cursor.getConfig().isAutoSkipNonRecoverableData() && exception instanceof NonRecoverableLedgerException) { log.warn("[{}][{}] read failed from ledger at position:{} : {}", cursor.ledger.getName(), cursor.getName(), @@ -153,7 +151,7 @@ private void internalReadEntriesFailed(ManagedLedgerException exception, Object } // fail callback if it couldn't find next valid ledger if (nexReadPosition == null) { - fail(exception, ctx); + fail(exception); return; } updateReadPosition(nexReadPosition); @@ -174,7 +172,7 @@ private void internalReadEntriesFailed(ManagedLedgerException exception, Object } } - fail(exception, ctx); + fail(exception); } } @@ -198,7 +196,7 @@ void checkReadCompletion() { try { cursor.readOperationCompleted(); } finally { - complete(ctx); + complete(); } } } @@ -255,7 +253,7 @@ public void recycle() { recyclerHandle.recycle(this); } - private void complete(Object ctx) { + private void complete() { cursor.ledger.getExecutor().execute(() -> { try { callback.readEntriesComplete(entries, ctx); @@ -266,7 +264,7 @@ private void complete(Object ctx) { }); } - private void fail(ManagedLedgerException e, Object ctx) { + private void fail(ManagedLedgerException e) { try { callback.readEntriesFailed(e, ctx); cursor.ledger.mbean.recordReadEntriesError(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 23b739987f4c9..2b494fccbe406 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -4650,7 +4650,7 @@ public void testReadEntriesWithSkipDeletedEntries() throws Exception { }) .when(ledger) .asyncReadEntry(Mockito.any(ReadHandle.class), Mockito.anyLong(), - Mockito.anyLong(), Mockito.any(), Mockito.any()); + Mockito.anyLong(), Mockito.any()); @Cleanup ManagedCursor cursor = ledger.openCursor("c"); @@ -4756,7 +4756,7 @@ public void testReadEntriesWithSkipDeletedEntriesAndWithSkipConditions() throws }) .when(ledger) .asyncReadEntry(Mockito.any(ReadHandle.class), Mockito.anyLong(), - Mockito.anyLong(), Mockito.any(), Mockito.any()); + Mockito.anyLong(), Mockito.any()); @Cleanup ManagedCursor cursor = ledger.openCursor("c"); From fea1e07207a36e4588eed62ad5cefa2eceb71cf3 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 18:29:07 +0800 Subject: [PATCH 09/16] Prevent recycle() is called without triggering callback --- .../mledger/ManagedLedgerException.java | 12 ++++ .../mledger/impl/ManagedCursorImpl.java | 6 +- .../bookkeeper/mledger/impl/OpReadEntry.java | 2 +- .../mledger/impl/ManagedCursorTest.java | 61 ------------------- ...entDispatcherMultipleConsumersClassic.java | 5 +- ...sistentDispatcherSingleActiveConsumer.java | 12 +--- 6 files changed, 20 insertions(+), 78 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java index 1fa565d6ec788..16fffeb88f37b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java @@ -135,6 +135,12 @@ public CursorAlreadyClosedException(String msg) { } } + public static class CancelledException extends ManagedLedgerException { + public CancelledException(String msg) { + super(msg); + } + } + public static class TooManyRequestsException extends ManagedLedgerException { public TooManyRequestsException(String msg) { super(msg); @@ -211,4 +217,10 @@ public synchronized Throwable fillInStackTrace() { // Disable stack traces to be filled in return null; } + + public static boolean shouldNotRead(ManagedLedgerException exception) { + return exception instanceof ConcurrentWaitCallbackException + || exception instanceof ManagedLedgerException.CursorAlreadyClosedException + || exception instanceof ManagedLedgerException.CancelledException; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index b5421988accd3..d50368257c9db 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1076,8 +1076,7 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re ctx, maxPosition, skipCondition, true); int opReadId = op.id; if (!WAITING_READ_OP_UPDATER.compareAndSet(this, null, op)) { - op.recycle(); - callback.readEntriesFailed(new ManagedLedgerException.ConcurrentWaitCallbackException(), ctx); + op.readEntriesFailed(new ManagedLedgerException.ConcurrentWaitCallbackException()); return; } @@ -1188,7 +1187,8 @@ public boolean cancelPendingReadRequest() { return null; }); if (op != null) { - op.recycle(); + final var msg = op.toString(); + op.readEntriesFailed(new ManagedLedgerException.CancelledException(msg + " is cancelled")); } return op != null && op != OpReadEntry.WAITING_READ_OP_FOR_CLOSED_CURSOR; } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index df8edc058b256..59b6c3bc138db 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -234,7 +234,7 @@ protected OpReadEntry newObject(Recycler.Handle recyclerHandle) { } }; - public void recycle() { + private void recycle() { if (recyclerHandle == null) { // This is the no-op instance, do not recycle return; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 2b494fccbe406..ade93a84abce3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -20,8 +20,6 @@ import static org.apache.bookkeeper.mledger.impl.EntryCountEstimator.estimateEntryCountByBytesSize; import static org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheImpl.BOOKKEEPER_READ_OVERHEAD_PER_ENTRY; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.eq; @@ -59,7 +57,6 @@ import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -74,7 +71,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; -import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; import lombok.Cleanup; @@ -123,7 +119,6 @@ import org.apache.pulsar.metadata.api.extended.SessionEvent; import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; import org.awaitility.Awaitility; -import org.mockito.MockedStatic; import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -4405,62 +4400,6 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { Awaitility.await().untilAsserted(() -> assertTrue(flag.get())); } - @Test - public void testOpReadEntryRecycle() throws Exception { - final Map opReadEntryToRecycleCount = new ConcurrentHashMap<>(); - final Supplier createOpReadEntry = () -> { - final OpReadEntry mockedOpReadEntry = mock(OpReadEntry.class); - doAnswer(__ -> opReadEntryToRecycleCount.computeIfAbsent(mockedOpReadEntry, - ignored -> new AtomicInteger(0)).getAndIncrement() - ).when(mockedOpReadEntry).recycle(); - return mockedOpReadEntry; - }; - - @Cleanup final MockedStatic mockedStaticOpReadEntry = Mockito.mockStatic(OpReadEntry.class); - mockedStaticOpReadEntry.when(() -> OpReadEntry.create(any(), any(), anyInt(), any(), - any(), any(), any(), anyBoolean())).thenAnswer(__ -> createOpReadEntry.get()); - - final ManagedLedgerConfig ledgerConfig = new ManagedLedgerConfig(); - ledgerConfig.setNewEntriesCheckDelayInMillis(10); - final ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", ledgerConfig); - final ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor("my_cursor"); - final List exceptions = new ArrayList<>(); - final AtomicBoolean readEntriesSuccess = new AtomicBoolean(false); - final ReadEntriesCallback callback = new ReadEntriesCallback() { - @Override - public void readEntriesComplete(List entries, Object ctx) { - readEntriesSuccess.set(true); - } - - @Override - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - exceptions.add(exception); - } - }; - - final int numReadRequests = 3; - for (int i = 0; i < numReadRequests; i++) { - cursor.asyncReadEntriesOrWait(1, callback, null, PositionFactory.create(0, 0)); - } - Awaitility.await().atMost(Duration.ofSeconds(1)) - .untilAsserted(() -> assertEquals(ledger.waitingCursors.size(), 1)); - assertTrue(cursor.cancelPendingReadRequest()); - - ledger.addEntry(new byte[1]); - Awaitility.await().atMost(Duration.ofSeconds(1)) - .untilAsserted(() -> assertTrue(ledger.waitingCursors.isEmpty())); - assertFalse(readEntriesSuccess.get()); - - assertEquals(exceptions.size(), numReadRequests - 1); - exceptions.forEach(e -> assertEquals(e.getMessage(), "We can only have a single waiting callback")); - assertEquals(opReadEntryToRecycleCount.size(), 3); - assertEquals(opReadEntryToRecycleCount.entrySet().stream() - .map(Map.Entry::getValue) - .map(AtomicInteger::get) - .collect(Collectors.toList()), - Arrays.asList(1, 1, 1)); - } - @Test public void testLazyCursorLedgerCreation() throws Exception { ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java index 0f496e461b85c..f8c5e2e21688e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumersClassic.java @@ -860,10 +860,9 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj ReadType readType = (ReadType) ctx; long waitTimeMillis = readFailureBackoff.next(); - // Do not keep reading more entries if the cursor is already closed. - if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (ManagedLedgerException.shouldNotRead(exception)) { if (log.isDebugEnabled()) { - log.debug("[{}] Cursor is already closed, skipping read more entries", cursor.getName()); + log.debug("[{}] skipping read more entries due to {}", cursor.getName(), exception.getMessage()); } // Set the wait time to -1 to avoid rescheduling the read. waitTimeMillis = -1; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index e809d984ae344..b1fa061530ff0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -35,7 +35,6 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.apache.bookkeeper.mledger.ManagedLedgerException.ConcurrentWaitCallbackException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NoMoreEntriesToReadException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.Position; @@ -469,20 +468,13 @@ private synchronized void internalReadEntriesFailed(ManagedLedgerException excep Consumer c = readEntriesCtx.getConsumer(); readEntriesCtx.recycle(); - // Do not keep reading messages from a closed cursor. - if (exception instanceof ManagedLedgerException.CursorAlreadyClosedException) { + if (ManagedLedgerException.shouldNotRead(exception)) { if (log.isDebugEnabled()) { - log.debug("[{}] Cursor was already closed, skipping read more entries", cursor.getName()); + log.debug("[{}] skipping read more entries due to {}", cursor.getName(), exception.getMessage()); } return; } - if (exception instanceof ConcurrentWaitCallbackException) { - // At most one pending read request is allowed when there are no more entries, we should not trigger more - // read operations in this case and just wait the existing read operation completes. - return; - } - long waitTimeMillis = readFailureBackoff.next(); if (exception instanceof NoMoreEntriesToReadException) { From 4bd6f5d35f52206e8482ecf2b94add3ebe9494ec Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 18:38:59 +0800 Subject: [PATCH 10/16] Don't complete OpReadEntry for exception that should not read --- .../org/apache/bookkeeper/mledger/impl/OpReadEntry.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 59b6c3bc138db..36a5febe7df41 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -22,6 +22,7 @@ import io.netty.util.Recycler.Handle; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; @@ -133,7 +134,12 @@ private void internalReadEntriesFailed(ManagedLedgerException exception) { if (!entries.isEmpty()) { // There were already some entries that were read before, we can return them - complete(); + if (ManagedLedgerException.shouldNotRead(exception)) { + entries.forEach(Entry::release); + fail(exception); + } else { + complete(); + } } else if (!cursor.isClosed() && cursor.getConfig().isAutoSkipNonRecoverableData() && exception instanceof NonRecoverableLedgerException) { log.warn("[{}][{}] read failed from ledger at position:{} : {}", cursor.ledger.getName(), cursor.getName(), From c99c5a3410b4c4dfdf5f8c8f277fcd2bdfb62141 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 18:44:27 +0800 Subject: [PATCH 11/16] Prevent multiple completion on OpReadEntry --- .../bookkeeper/mledger/impl/OpReadEntry.java | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 36a5febe7df41..b9881aae722f0 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -22,7 +22,6 @@ import io.netty.util.Recycler.Handle; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntriesCallback; @@ -57,6 +56,7 @@ class OpReadEntry { Predicate skipCondition; boolean skipOpenLedgerFullyAcked = false; + boolean completed = false; public static OpReadEntry create(ManagedCursorImpl cursor, Position readPositionRef, int count, ReadEntriesCallback callback, Object ctx, Position maxPosition, @@ -77,6 +77,7 @@ public static OpReadEntry create(ManagedCursorImpl cursor, Position readPosition op.skipOpenLedgerFullyAcked = skipOpenLedgerFullyAcked; op.ctx = ctx; op.nextReadPosition = PositionFactory.create(op.readPosition); + op.completed = true; return op; } @@ -113,6 +114,13 @@ private void internalReadEntriesComplete(List returnedEntries) { public void readEntriesComplete(List returnedEntries) { try { + synchronized (this) { + if (completed) { + return; + } + completed = true; + returnedEntries.forEach(Entry::release); + } internalReadEntriesComplete(returnedEntries); } catch (Throwable throwable) { log.error("[{}] Fallback to readEntriesFailed for exception in readEntriesComplete", this, throwable); @@ -122,6 +130,12 @@ public void readEntriesComplete(List returnedEntries) { public void readEntriesFailed(ManagedLedgerException exception) { try { + synchronized (this) { + if (completed) { + return; + } + completed = true; + } internalReadEntriesFailed(exception); } catch (Throwable throwable) { // At least we should complete the callback @@ -195,7 +209,10 @@ void checkReadCompletion() { // We still have more entries to read from the next ledger, schedule a new async operation cursor.ledger.getExecutor().execute(() -> { readPosition = cursor.ledger.startReadOperationOnLedger(nextReadPosition); - cursor.ledger.asyncReadEntries(OpReadEntry.this); + synchronized (this) { + completed = false; + cursor.ledger.asyncReadEntries(OpReadEntry.this); + } }); } else { // The reading was already completed, release resources and trigger callback @@ -256,6 +273,7 @@ private void recycle() { maxPosition = null; skipCondition = null; skipOpenLedgerFullyAcked = false; + completed = false; recyclerHandle.recycle(this); } From f5b9cf10e41f968fdd526d953879cdf8d7192a69 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 20:23:55 +0800 Subject: [PATCH 12/16] Fix OpReadEntry completed state --- .../java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index b9881aae722f0..0b690cfb65cd8 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -77,7 +77,7 @@ public static OpReadEntry create(ManagedCursorImpl cursor, Position readPosition op.skipOpenLedgerFullyAcked = skipOpenLedgerFullyAcked; op.ctx = ctx; op.nextReadPosition = PositionFactory.create(op.readPosition); - op.completed = true; + op.completed = false; return op; } @@ -116,10 +116,10 @@ public void readEntriesComplete(List returnedEntries) { try { synchronized (this) { if (completed) { + returnedEntries.forEach(Entry::release); return; } completed = true; - returnedEntries.forEach(Entry::release); } internalReadEntriesComplete(returnedEntries); } catch (Throwable throwable) { From 68bf61ebf4bcaa00f98678f88975d56f24e11133 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 21:44:06 +0800 Subject: [PATCH 13/16] Remove warn logs for cancellation --- .../apache/bookkeeper/mledger/impl/OpReadEntry.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index 0b690cfb65cd8..bdbb8b9ff17f3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -145,15 +145,15 @@ public void readEntriesFailed(ManagedLedgerException exception) { private void internalReadEntriesFailed(ManagedLedgerException exception) { cursor.readOperationCompleted(); + if (ManagedLedgerException.shouldNotRead(exception)) { + entries.forEach(Entry::release); + fail(exception); + return; + } if (!entries.isEmpty()) { // There were already some entries that were read before, we can return them - if (ManagedLedgerException.shouldNotRead(exception)) { - entries.forEach(Entry::release); - fail(exception); - } else { - complete(); - } + complete(); } else if (!cursor.isClosed() && cursor.getConfig().isAutoSkipNonRecoverableData() && exception instanceof NonRecoverableLedgerException) { log.warn("[{}][{}] read failed from ledger at position:{} : {}", cursor.ledger.getName(), cursor.getName(), From 0e435bf03cd53e51ca3d2e9e003db565f3f9cdfa Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 21:51:33 +0800 Subject: [PATCH 14/16] Fix testReadWithError --- .../bookkeeper/mledger/impl/cache/PendingReadsManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java index 1d5a6cc74d066..23325f650c4ae 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/PendingReadsManager.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl.cache; +import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; import com.google.common.annotations.VisibleForTesting; import io.prometheus.client.Counter; import java.util.ArrayList; @@ -32,6 +33,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.EntryImpl; /** @@ -292,7 +294,8 @@ private void readEntriesComplete(List callbacks, // this method isn't synchronized since that could lead to deadlocks private void readEntriesFailed(List callbacks, Throwable error) { for (ReadEntriesCallbackWithContext callback : callbacks) { - callback.future.completeExceptionally(error); + ManagedLedgerException mlException = createManagedLedgerException(error); + callback.future.completeExceptionally(mlException); } } From c2efb102f157544b698e9d6d461830a760761980 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 22:49:40 +0800 Subject: [PATCH 15/16] Simplify the code --- .../mledger/impl/ManagedLedgerImpl.java | 62 +++++++------------ .../mledger/util/CallbackUtils.java | 46 -------------- 2 files changed, 22 insertions(+), 86 deletions(-) delete mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 5a54504c85ae8..86831318734b4 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -130,7 +130,6 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext; import org.apache.bookkeeper.mledger.util.CallbackMutex; -import org.apache.bookkeeper.mledger.util.CallbackUtils; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; import org.apache.bookkeeper.net.BookieId; @@ -320,9 +319,6 @@ public boolean isFenced() { protected final ManagedLedgerMBeanImpl mbean; protected final Clock clock; - private static final AtomicLongFieldUpdater READ_OP_COUNT_UPDATER = AtomicLongFieldUpdater - .newUpdater(ManagedLedgerImpl.class, "readOpCount"); - private volatile long readOpCount = 0; protected static final AtomicLongFieldUpdater ADD_OP_COUNT_UPDATER = AtomicLongFieldUpdater .newUpdater(ManagedLedgerImpl.class, "addOpCount"); private volatile long addOpCount = 0; @@ -2396,53 +2392,39 @@ private void internalReadFromLedger(ReadHandle ledger, OpReadEntry opReadEntry) protected void asyncReadEntry(ReadHandle ledger, Position position, ReadEntryCallback callback, Object ctx) { mbean.addEntriesRead(1); + final var future = entryCache.asyncReadEntry(ledger, position); + future.whenComplete((entry, throwable) -> { + if (throwable == null) { + callback.readEntryComplete(entry, ctx); + } else { + callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); + } + }); if (config.getReadEntryTimeoutSeconds() > 0) { - long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); long createdTime = System.nanoTime(); - final var future = entryCache.asyncReadEntry(ledger, position).thenApply(List::of); - future.whenComplete((entries, throwable) -> { - if (throwable == null) { - callback.readEntryComplete(entries.get(0), ctx); - } else { - callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); - } - }); lastReadEntriesOp = new PendingReadEntriesOp(position.getLedgerId(), position.getEntryId(), createdTime, - readOpCount, future); - } else { - entryCache.asyncReadEntry(ledger, position).whenComplete((entry, throwable) -> - CallbackUtils.complete(entry, throwable, callback, ctx)); + future); } } protected void asyncReadEntry(ReadHandle ledger, long firstEntry, long lastEntry, OpReadEntry opReadEntry) { IntSupplier expectedReadCount = opReadEntry.cursor::getNumberOfCursorsAtSamePositionOrBefore; + final var future = entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount); + future.whenComplete((entries, throwable) -> { + if (throwable == null) { + opReadEntry.readEntriesComplete(entries); + } else { + opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable)); + } + }); if (config.getReadEntryTimeoutSeconds() > 0) { - long readOpCount = READ_OP_COUNT_UPDATER.incrementAndGet(this); long createdTime = System.nanoTime(); - final var future = entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount); - future.whenComplete((entries, throwable) -> { - if (throwable == null) { - opReadEntry.readEntriesComplete(entries); - } else { - opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable)); - } - }); - lastReadEntriesOp = new PendingReadEntriesOp(ledger.getId(), firstEntry, createdTime, readOpCount, future); - } else { - entryCache.asyncReadEntry(ledger, firstEntry, lastEntry, expectedReadCount).whenComplete( - (entries, throwable) -> { - if (throwable == null) { - opReadEntry.readEntriesComplete(entries); - } else { - opReadEntry.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable)); - } - }); + lastReadEntriesOp = new PendingReadEntriesOp(ledger.getId(), firstEntry, createdTime, future); } } - record PendingReadEntriesOp(long ledgerId, long entryId, long createdTime, long readOpCount, - CompletableFuture> future) { + record PendingReadEntriesOp(long ledgerId, long entryId, long createdTime, + CompletableFuture future) { } @Override @@ -4425,8 +4407,8 @@ private void checkReadTimeout() { return; } final var lastReadEntriesOp = this.lastReadEntriesOp; - if (lastReadEntriesOp != null && lastReadEntriesOp.readOpCount > 0 && TimeUnit.NANOSECONDS.toSeconds( - System.nanoTime() - lastReadEntriesOp.createdTime) >= timeoutSec) { + if (lastReadEntriesOp != null && TimeUnit.NANOSECONDS.toSeconds( + System.nanoTime() - lastReadEntriesOp.createdTime) >= timeoutSec) { log.warn("[{}]-{}-{} read entry timeout after {} sec", this.name, lastReadEntriesOp.ledgerId, lastReadEntriesOp.entryId, timeoutSec); lastReadEntriesOp.future.completeExceptionally(createManagedLedgerException( diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java deleted file mode 100644 index 760cb10f905f1..0000000000000 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/CallbackUtils.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.bookkeeper.mledger.util; - -import java.util.List; -import org.apache.bookkeeper.mledger.AsyncCallbacks; -import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedLedgerException; -import org.jspecify.annotations.Nullable; - -public class CallbackUtils { - - public static void complete(List entries, @Nullable Throwable throwable, - AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { - if (throwable == null) { - callback.readEntriesComplete(entries, ctx); - } else { - callback.readEntriesFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); - } - } - - public static void complete(Entry entry, @Nullable Throwable throwable, AsyncCallbacks.ReadEntryCallback callback, - Object ctx) { - if (throwable == null) { - callback.readEntryComplete(entry, ctx); - } else { - callback.readEntryFailed(ManagedLedgerException.getManagedLedgerException(throwable), ctx); - } - } -} From 8665959390d7dd9fcb322040f7b1b40302064261 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Sep 2025 22:56:14 +0800 Subject: [PATCH 16/16] Revert completed design --- .../bookkeeper/mledger/impl/OpReadEntry.java | 21 +------------------ 1 file changed, 1 insertion(+), 20 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index bdbb8b9ff17f3..db319c38ecfef 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -56,7 +56,6 @@ class OpReadEntry { Predicate skipCondition; boolean skipOpenLedgerFullyAcked = false; - boolean completed = false; public static OpReadEntry create(ManagedCursorImpl cursor, Position readPositionRef, int count, ReadEntriesCallback callback, Object ctx, Position maxPosition, @@ -77,7 +76,6 @@ public static OpReadEntry create(ManagedCursorImpl cursor, Position readPosition op.skipOpenLedgerFullyAcked = skipOpenLedgerFullyAcked; op.ctx = ctx; op.nextReadPosition = PositionFactory.create(op.readPosition); - op.completed = false; return op; } @@ -114,13 +112,6 @@ private void internalReadEntriesComplete(List returnedEntries) { public void readEntriesComplete(List returnedEntries) { try { - synchronized (this) { - if (completed) { - returnedEntries.forEach(Entry::release); - return; - } - completed = true; - } internalReadEntriesComplete(returnedEntries); } catch (Throwable throwable) { log.error("[{}] Fallback to readEntriesFailed for exception in readEntriesComplete", this, throwable); @@ -130,12 +121,6 @@ public void readEntriesComplete(List returnedEntries) { public void readEntriesFailed(ManagedLedgerException exception) { try { - synchronized (this) { - if (completed) { - return; - } - completed = true; - } internalReadEntriesFailed(exception); } catch (Throwable throwable) { // At least we should complete the callback @@ -209,10 +194,7 @@ void checkReadCompletion() { // We still have more entries to read from the next ledger, schedule a new async operation cursor.ledger.getExecutor().execute(() -> { readPosition = cursor.ledger.startReadOperationOnLedger(nextReadPosition); - synchronized (this) { - completed = false; - cursor.ledger.asyncReadEntries(OpReadEntry.this); - } + cursor.ledger.asyncReadEntries(OpReadEntry.this); }); } else { // The reading was already completed, release resources and trigger callback @@ -273,7 +255,6 @@ private void recycle() { maxPosition = null; skipCondition = null; skipOpenLedgerFullyAcked = false; - completed = false; recyclerHandle.recycle(this); }