Skip to content

Commit 7daad7a

Browse files
Demogorgon314lhotari
authored andcommitted
[fix][ml] Managed ledger should recover after open ledger failed (apache#23368)
(cherry picked from commit 77cb67a) (cherry picked from commit c80eb40)
1 parent b97c18f commit 7daad7a

File tree

2 files changed

+32
-0
lines changed

2 files changed

+32
-0
lines changed

managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -401,6 +401,7 @@ public void initializeFailed(ManagedLedgerException e) {
401401

402402
// Clean the map if initialization fails
403403
ledgers.remove(name, future);
404+
entryCacheManager.removeEntryCache(name);
404405

405406
if (pendingInitializeLedgers.remove(name, pendingLedger)) {
406407
pendingLedger.ledger.asyncClose(new CloseCallback() {

managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerErrorsTest.java

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,12 +31,14 @@
3131
import java.util.concurrent.CompletableFuture;
3232
import java.util.concurrent.CountDownLatch;
3333
import java.util.concurrent.ExecutionException;
34+
import java.util.concurrent.TimeUnit;
3435
import java.util.concurrent.atomic.AtomicReference;
3536
import lombok.Cleanup;
3637
import org.apache.bookkeeper.client.BKException;
3738
import org.apache.bookkeeper.client.BookKeeper;
3839
import org.apache.bookkeeper.client.LedgerHandle;
3940
import org.apache.bookkeeper.client.api.DigestType;
41+
import org.apache.bookkeeper.mledger.AsyncCallbacks;
4042
import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback;
4143
import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback;
4244
import org.apache.bookkeeper.mledger.Entry;
@@ -509,6 +511,35 @@ public void recoverAfterWriteError() throws Exception {
509511
entries.forEach(Entry::release);
510512
}
511513

514+
@Test
515+
public void recoverAfterOpenManagedLedgerFail() throws Exception {
516+
ManagedLedger ledger = factory.open("recoverAfterOpenManagedLedgerFail");
517+
Position position = ledger.addEntry("entry".getBytes());
518+
ledger.close();
519+
bkc.failAfter(0, BKException.Code.BookieHandleNotAvailableException);
520+
try {
521+
factory.open("recoverAfterOpenManagedLedgerFail");
522+
} catch (Exception e) {
523+
// ok
524+
}
525+
526+
ledger = factory.open("recoverAfterOpenManagedLedgerFail");
527+
CompletableFuture<byte[]> future = new CompletableFuture<>();
528+
((ManagedLedgerImpl)ledger).asyncReadEntry((PositionImpl)position, new AsyncCallbacks.ReadEntryCallback() {
529+
@Override
530+
public void readEntryComplete(Entry entry, Object ctx) {
531+
future.complete(entry.getData());
532+
}
533+
534+
@Override
535+
public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
536+
future.completeExceptionally(exception);
537+
}
538+
}, null);
539+
byte[] bytes = future.get(30, TimeUnit.SECONDS);
540+
assertEquals(new String(bytes), "entry");
541+
}
542+
512543
@Test
513544
public void recoverLongTimeAfterMultipleWriteErrors() throws Exception {
514545
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("recoverLongTimeAfterMultipleWriteErrors");

0 commit comments

Comments
 (0)