Skip to content

Commit

Permalink
KAFKA-17634: Tweak wakeup logic to match WakeupTrigger changes
Browse files Browse the repository at this point in the history
  • Loading branch information
AndrewJSchofield committed Sep 27, 2024
1 parent e1deeb4 commit bc32e28
Show file tree
Hide file tree
Showing 2 changed files with 45 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -492,7 +492,7 @@ ShareFetchCollector<K, V> build(
public Set<String> subscription() {
acquireAndEnsureOpen();
try {
return subscriptions.subscription();
return Collections.unmodifiableSet(subscriptions.subscription());
} finally {
release();
}
Expand Down Expand Up @@ -594,7 +594,6 @@ public synchronized ConsumerRecords<K, V> poll(final Duration timeout) {
return ConsumerRecords.empty();
} finally {
kafkaShareConsumerMetrics.recordPollEnd(timer.currentTimeMs());
wakeupTrigger.clearTask();
release();
}
}
Expand All @@ -612,13 +611,16 @@ private ShareFetch<K, V> pollForFetches(final Timer timer) {

// 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) {
log.trace("Timeout during fetch", e);
throw e;
} finally {
timer.update(pollTimer.currentTimeMs());
wakeupTrigger.clearTask();
}

return collect(Collections.emptyMap());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,18 +53,21 @@ public void wakeup() {
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;
}
});
}

/**
* 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
Expand Down Expand Up @@ -105,6 +108,25 @@ public void setFetchAction(final FetchBuffer fetchBuffer) {
}
}

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());
}
Expand All @@ -113,7 +135,7 @@ public void clearTask() {
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;
Expand Down Expand Up @@ -172,4 +194,17 @@ public FetchBuffer fetchBuffer() {
return fetchBuffer;
}
}

static class ShareFetchAction implements Wakeupable {

private final ShareFetchBuffer fetchBuffer;

public ShareFetchAction(ShareFetchBuffer fetchBuffer) {
this.fetchBuffer = fetchBuffer;
}

public ShareFetchBuffer fetchBuffer() {
return fetchBuffer;
}
}
}

0 comments on commit bc32e28

Please sign in to comment.