Skip to content

Commit

Permalink
KAFKA-16577: New consumer fails with stop within allotted timeout in …
Browse files Browse the repository at this point in the history
…consumer_test.py system test (apache#15784)

The KafkaAsyncConsumer would occasionally fail to stop when wakeup() was invoked. It turns out that there's a race condition between the thread that invokes wakeup() and the thread that is performing an action on the consumer. If the operation's Future is already completed by thread A when thread B invoke's completeExceptionally() inside wakeup(), the WakeupException will be ignored. We should use the return value from completeExceptionally() to determine if that call actually triggered completion of the Future. If that method returns false, that signals that the Future was already completed, and the exception we passed to completeExceptionally() was ignored. Therefore, we then need to return a new WakeupFuture instead of null so that the next call to setActiveTask() will throw the WakeupException.

Reviewers: Lucas Brutschy <[email protected]>
  • Loading branch information
kirktrue authored May 15, 2024
1 parent c218c4e commit 3f8d11f
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,12 @@ public void wakeup() {
return new WakeupFuture();
} else if (task instanceof ActiveFuture) {
ActiveFuture active = (ActiveFuture) task;
active.future().completeExceptionally(new WakeupException());
return null;
boolean wasTriggered = active.future().completeExceptionally(new WakeupException());

// If the Future was *already* completed when we invoke completeExceptionally, the WakeupException
// will be ignored. If it was already completed, we then need to return a new WakeupFuture so that the
// next call to setActiveTask will throw the WakeupException.
return wasTriggered ? null : new WakeupFuture();
} else if (task instanceof FetchAction) {
FetchAction fetchAction = (FetchAction) task;
fetchAction.fetchBuffer().wakeup();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
Expand Down Expand Up @@ -171,6 +172,39 @@ public void testDisableWakeupPreservedByClearTask() {
assertDoesNotThrow(() -> wakeupTrigger.maybeTriggerWakeup());
}

@Test
public void testExceptionTriggeredWhenTaskAsynchronouslyCompleted() {
final CompletableFuture<Void> task = new CompletableFuture<>();
wakeupTrigger.setActiveTask(task);
task.complete(null);
wakeupTrigger.wakeup();
assertNotNull(wakeupTrigger.getPendingTask());
assertInstanceOf(WakeupTrigger.WakeupFuture.class, wakeupTrigger.getPendingTask());
assertThrows(WakeupException.class, () -> wakeupTrigger.maybeTriggerWakeup());
}

@Test
public void testExceptionTriggeredWhenTaskAsynchronouslyFailed() {
final CompletableFuture<Void> task = new CompletableFuture<>();
wakeupTrigger.setActiveTask(task);
task.completeExceptionally(new RuntimeException("Simulated error"));
wakeupTrigger.wakeup();
assertNotNull(wakeupTrigger.getPendingTask());
assertInstanceOf(WakeupTrigger.WakeupFuture.class, wakeupTrigger.getPendingTask());
assertThrows(WakeupException.class, () -> wakeupTrigger.maybeTriggerWakeup());
}

@Test
public void testExceptionTriggeredWhenTaskAsynchronouslyCancelled() {
final CompletableFuture<Void> task = new CompletableFuture<>();
wakeupTrigger.setActiveTask(task);
task.cancel(true);
wakeupTrigger.wakeup();
assertNotNull(wakeupTrigger.getPendingTask());
assertInstanceOf(WakeupTrigger.WakeupFuture.class, wakeupTrigger.getPendingTask());
assertThrows(WakeupException.class, () -> wakeupTrigger.maybeTriggerWakeup());
}

private void assertWakeupExceptionIsThrown(final CompletableFuture<?> future) {
assertTrue(future.isCompletedExceptionally());
assertInstanceOf(WakeupException.class,
Expand Down

0 comments on commit 3f8d11f

Please sign in to comment.