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] Topic load timeout due to ml data ledger future never finishes #23772

Merged
merged 3 commits into from
Dec 25, 2024
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 @@ -574,6 +574,7 @@ public void operationFailed(MetaStoreException e) {
executor.execute(() -> {
mbean.endDataLedgerCreateOp();
if (rc != BKException.Code.OK) {
log.error("[{}] Error creating ledger rc={} {}", name, rc, BKException.getMessage(rc));
callback.initializeFailed(createManagedLedgerException(rc));
return;
}
Expand Down Expand Up @@ -4141,7 +4142,7 @@ public Clock getClock() {
protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object ctx) {
if (ctx instanceof CompletableFuture) {
// ledger-creation is already timed out and callback is already completed so, delete this ledger and return.
if (((CompletableFuture) ctx).complete(lh)) {
if (((CompletableFuture) ctx).complete(lh) || rc == BKException.Code.TimeoutException) {
return false;
} else {
if (rc == BKException.Code.OK) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4236,6 +4236,31 @@ public void testNonDurableCursorCreateForInactiveLedger() throws Exception {
assertNotNull(ml.newNonDurableCursor(Position));
}

@Test(timeOut = 60 * 1000)
public void testCreateDataLedgerTimeout() throws Exception {
String mlName = UUID.randomUUID().toString();
ManagedLedgerFactoryImpl factory = null;
ManagedLedger ml = null;
try {
factory = new ManagedLedgerFactoryImpl(metadataStore, bkc);
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setMetadataOperationsTimeoutSeconds(5);
bkc.delay(10 * 1000);
ml = factory.open(mlName, config);
fail("Should get a timeout ex");
} catch (ManagedLedgerException ex) {
assertTrue(ex.getMessage().contains("timeout"));
} finally {
// cleanup.
if (ml != null) {
ml.delete();
}
if (factory != null) {
factory.shutdown();
}
}
}

/***
* When a ML tries to create a ledger, it will create a delay task to check if the ledger create request is timeout.
* But we should guarantee that the delay task should be canceled after the ledger create request responded.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import static com.google.common.base.Preconditions.checkArgument;
import com.google.common.collect.Lists;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
Expand All @@ -35,6 +36,7 @@
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
Expand Down Expand Up @@ -70,6 +72,7 @@ public class PulsarMockBookKeeper extends BookKeeper {

final OrderedExecutor orderedExecutor;
final ExecutorService executor;
final ScheduledExecutorService scheduler;

@Override
public ClientConfiguration getConf() {
Expand Down Expand Up @@ -97,6 +100,7 @@ public static Collection<BookieId> getMockEnsemble() {
public PulsarMockBookKeeper(OrderedExecutor orderedExecutor) throws Exception {
this.orderedExecutor = orderedExecutor;
this.executor = orderedExecutor.chooseThread();
scheduler = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("mock-bk-scheduler"));
}

@Override
Expand Down Expand Up @@ -290,7 +294,7 @@ public void shutdown() {
for (PulsarMockLedgerHandle ledger : ledgers.values()) {
ledger.entries.clear();
}

scheduler.shutdown();
ledgers.clear();
}

Expand Down Expand Up @@ -331,6 +335,15 @@ synchronized CompletableFuture<Void> getProgrammedFailure() {
return failures.isEmpty() ? defaultResponse : failures.remove(0);
}

public void delay(long millis) {
CompletableFuture<Void> delayFuture = new CompletableFuture<>();
scheduler.schedule(() -> {
delayFuture.complete(null);
}, millis, TimeUnit.MILLISECONDS);
failures.add(delayFuture);
}


public void failNow(int rc) {
failAfter(0, rc);
}
Expand Down
Loading