Skip to content

Commit

Permalink
issue 4109: Fix Flaky-test: HandleFailuresTest.testHandleFailureBooki…
Browse files Browse the repository at this point in the history
…eNotInWriteSet (apache#4110)

Master Issue: apache#4109
### Motivation

Fix Flaky-test: HandleFailuresTest.testHandleFailureBookieNotInWriteSet

When we call `b1Delay.completeExceptionally(new BKException.BKWriteException())` at line480(code1),  the `preWriteHook` will complete with exception  and then do some actions in the choosen thread(code2), e.g., put the failure bookie to `delayedWriteFailedBookies`(code3).  So the `delayedWriteFailedBookies` update is async.

However, `lh.appendAsync("entry2".getBytes()))`(Line483 in Code4) is invoked in main thread. So when `appendAsync` execute,  the `delayedWriteFailedBookies` could be not updated yet, and `changeInProgress.complete(null)`(code5) will never be invoked.
  • Loading branch information
AnonHxy authored Dec 4, 2023
1 parent c1b7f76 commit b3662b7
Show file tree
Hide file tree
Showing 2 changed files with 13 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -31,11 +31,13 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.client.api.WriteFlag;
import org.apache.bookkeeper.common.concurrent.FutureUtils;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.proto.MockBookieClient;
import org.apache.bookkeeper.versioning.Versioned;
import org.junit.Assert;
import org.junit.Test;
Expand Down Expand Up @@ -480,14 +482,20 @@ public void testHandleFailureBookieNotInWriteSet() throws Exception {
b1Delay.completeExceptionally(new BKException.BKWriteException());

log.info("write second entry, should have enough bookies, but blocks completion on failure handling");
CompletableFuture<?> e2 = lh.appendAsync("entry2".getBytes());
AtomicReference<CompletableFuture<?>> e2 = new AtomicReference<>();

// Execute appendAsync at the same thread of preWriteHook exception thread. So that the
// `delayedWriteFailedBookies` could update before appendAsync invoke.
((MockBookieClient) clientCtx.getBookieClient()).getExecutor()
.chooseThread(lh.ledgerId)
.execute(() -> e2.set(lh.appendAsync("entry2".getBytes())));
changeInProgress.get();
assertEventuallyTrue("e2 should eventually complete", () -> lh.pendingAddOps.peek().completed);
Assert.assertFalse("e2 shouldn't be completed to client", e2.isDone());
Assert.assertFalse("e2 shouldn't be completed to client", e2.get().isDone());
blockEnsembleChange.complete(null); // allow ensemble change to continue

log.info("e2 should complete");
e2.get(10, TimeUnit.SECONDS);
e2.get().get(10, TimeUnit.SECONDS);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import lombok.Getter;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.api.WriteFlag;
import org.apache.bookkeeper.common.concurrent.FutureUtils;
Expand All @@ -56,6 +57,7 @@
public class MockBookieClient implements BookieClient {
static final Logger LOG = LoggerFactory.getLogger(MockBookieClient.class);

@Getter
final OrderedExecutor executor;
final MockBookies mockBookies;
final Set<BookieId> errorBookies =
Expand Down

0 comments on commit b3662b7

Please sign in to comment.