KAFKA-17634 Tweak wakeup logic to match WakeupTrigger changes (#17304)

WakeupTrigger was refactored as a result of changes in AsyncKafkaConsumer. This PR makes the equivalent changes in ShareConsumerImpl.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Andrew Schofield 2024-09-30 14:50:44 +01:00 committed by GitHub
parent e27d0dfb17
commit 800de133bf
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 97 additions and 8 deletions

View File

@ -492,7 +492,7 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
public Set<String> subscription() {
acquireAndEnsureOpen();
try {
return subscriptions.subscription();
return Collections.unmodifiableSet(subscriptions.subscription());
} finally {
release();
}
@ -594,7 +594,6 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
return ConsumerRecords.empty();
} finally {
kafkaShareConsumerMetrics.recordPollEnd(timer.currentTimeMs());
wakeupTrigger.clearTask();
release();
}
}
@ -612,6 +611,8 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
// Wait a bit - this is where we will fetch records
Timer pollTimer = time.timer(pollTimeout);
wakeupTrigger.setShareFetchAction(fetchBuffer);
try {
fetchBuffer.awaitNotEmpty(pollTimer);
} catch (InterruptException e) {
@ -619,6 +620,7 @@ public class ShareConsumerImpl<K, V> implements ShareConsumerDelegate<K, V> {
throw e;
} finally {
timer.update(pollTimer.currentTimeMs());
wakeupTrigger.clearTask();
}
return collect(Collections.emptyMap());

View File

@ -53,6 +53,10 @@ public class WakeupTrigger {
FetchAction fetchAction = (FetchAction) task;
fetchAction.fetchBuffer().wakeup();
return new WakeupFuture();
} else if (task instanceof ShareFetchAction) {
ShareFetchAction shareFetchAction = (ShareFetchAction) task;
shareFetchAction.fetchBuffer().wakeup();
return new WakeupFuture();
} else {
return task;
}
@ -60,11 +64,10 @@ public class WakeupTrigger {
}
/**
* If there is no pending task, set the pending task active.
* If wakeup was called before setting an active task, the current task will complete exceptionally with
* WakeupException right
* away.
* if there is an active task, throw exception.
* If there is no pending task, set the pending task active.
* If wakeup was called before setting an active task, the current task will complete exceptionally with
* WakeupException right away.
* If there is an active task, throw exception.
* @param currentTask
* @param <T>
* @return
@ -105,6 +108,25 @@ public class WakeupTrigger {
}
}
public void setShareFetchAction(final ShareFetchBuffer fetchBuffer) {
final AtomicBoolean throwWakeupException = new AtomicBoolean(false);
pendingTask.getAndUpdate(task -> {
if (task == null) {
return new ShareFetchAction(fetchBuffer);
} else if (task instanceof WakeupFuture) {
throwWakeupException.set(true);
return null;
} else if (task instanceof DisabledWakeups) {
return task;
}
// last active state is still active
throw new IllegalStateException("Last active task is still active");
});
if (throwWakeupException.get()) {
throw new WakeupException();
}
}
public void disableWakeups() {
pendingTask.set(new DisabledWakeups());
}
@ -113,7 +135,7 @@ public class WakeupTrigger {
pendingTask.getAndUpdate(task -> {
if (task == null) {
return null;
} else if (task instanceof ActiveFuture || task instanceof FetchAction) {
} else if (task instanceof ActiveFuture || task instanceof FetchAction || task instanceof ShareFetchAction) {
return null;
}
return task;
@ -172,4 +194,17 @@ public class WakeupTrigger {
return fetchBuffer;
}
}
static class ShareFetchAction implements Wakeupable {
private final ShareFetchBuffer fetchBuffer;
public ShareFetchAction(ShareFetchBuffer fetchBuffer) {
this.fetchBuffer = fetchBuffer;
}
public ShareFetchBuffer fetchBuffer() {
return fetchBuffer;
}
}
}

View File

@ -191,6 +191,45 @@ public class ShareConsumerImplTest {
assertDoesNotThrow(() -> consumer.poll(Duration.ZERO));
}
@Test
public void testWakeupAfterEmptyFetch() {
consumer = newConsumer();
final String topicName = "foo";
final int partition = 3;
doAnswer(invocation -> {
consumer.wakeup();
return ShareFetch.empty();
}).doAnswer(invocation -> ShareFetch.empty()).when(fetchCollector).collect(any(ShareFetchBuffer.class));
consumer.subscribe(singletonList(topicName));
assertThrows(WakeupException.class, () -> consumer.poll(Duration.ofMinutes(1)));
assertDoesNotThrow(() -> consumer.poll(Duration.ZERO));
}
@Test
public void testWakeupAfterNonEmptyFetch() {
consumer = newConsumer();
final String topicName = "foo";
final int partition = 3;
final TopicIdPartition tip = new TopicIdPartition(Uuid.randomUuid(), partition, topicName);
final ShareInFlightBatch<String, String> batch = new ShareInFlightBatch<>(tip);
batch.addRecord(new ConsumerRecord<>(topicName, partition, 2, "key1", "value1"));
doAnswer(invocation -> {
consumer.wakeup();
final ShareFetch<String, String> fetch = ShareFetch.empty();
fetch.add(tip, batch);
return fetch;
}).when(fetchCollector).collect(Mockito.any(ShareFetchBuffer.class));
consumer.subscribe(singletonList(topicName));
// since wakeup() is called when the non-empty fetch is returned the wakeup should be ignored
assertDoesNotThrow(() -> consumer.poll(Duration.ofMinutes(1)));
// the previously ignored wake-up should not be ignored in the next call
assertThrows(WakeupException.class, () -> consumer.poll(Duration.ZERO));
}
@Test
public void testFailOnClosedConsumer() {
consumer = newConsumer();

View File

@ -110,6 +110,19 @@ public class WakeupTriggerTest {
}
}
@Test
public void testWakeupFromShareFetchAction() {
try (final ShareFetchBuffer fetchBuffer = mock(ShareFetchBuffer.class)) {
wakeupTrigger.setShareFetchAction(fetchBuffer);
wakeupTrigger.wakeup();
verify(fetchBuffer).wakeup();
final WakeupTrigger.Wakeupable wakeupable = wakeupTrigger.getPendingTask();
assertInstanceOf(WakeupTrigger.WakeupFuture.class, wakeupable);
}
}
@Test
public void testManualTriggerWhenWakeupCalled() {
wakeupTrigger.wakeup();