Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix][ml] Managed ledger should recover after open ledger failed #23368

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -440,6 +440,7 @@ public void initializeFailed(ManagedLedgerException e) {

// Clean the map if initialization fails
ledgers.remove(name, future);
entryCacheManager.removeEntryCache(name);

if (pendingInitializeLedgers.remove(name, pendingLedger)) {
pendingLedger.ledger.asyncClose(new CloseCallback() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,12 +31,14 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import lombok.Cleanup;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.DigestType;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback;
import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback;
import org.apache.bookkeeper.mledger.Entry;
Expand Down Expand Up @@ -509,6 +511,35 @@ public void recoverAfterWriteError() throws Exception {
entries.forEach(Entry::release);
}

@Test
public void recoverAfterOpenManagedLedgerFail() throws Exception {
ManagedLedger ledger = factory.open("recoverAfterOpenManagedLedgerFail");
Position position = ledger.addEntry("entry".getBytes());
ledger.close();
bkc.failAfter(0, BKException.Code.BookieHandleNotAvailableException);
try {
factory.open("recoverAfterOpenManagedLedgerFail");
} catch (Exception e) {
// ok
}

ledger = factory.open("recoverAfterOpenManagedLedgerFail");
CompletableFuture<byte[]> future = new CompletableFuture<>();
ledger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() {
@Override
public void readEntryComplete(Entry entry, Object ctx) {
future.complete(entry.getData());
}

@Override
public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
future.completeExceptionally(exception);
}
}, null);
byte[] bytes = future.get(30, TimeUnit.SECONDS);
assertEquals(new String(bytes), "entry");
}

@Test
public void recoverLongTimeAfterMultipleWriteErrors() throws Exception {
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("recoverLongTimeAfterMultipleWriteErrors");
Expand Down
Loading