-
Notifications
You must be signed in to change notification settings - Fork 14.7k
KAFKA-19694: Trigger StreamsRebalanceListener in Consumer.close #20511
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
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pull Request Overview
This PR implements a StreamsRebalanceListener invoker to trigger rebalance callbacks when the consumer closes, ensuring that Streams applications can properly handle state cleanup when losing tasks due to fencing or other failures.
- Adds
StreamsRebalanceListenerInvoker
class to manage and invoke streams rebalance callbacks with proper error handling - Updates consumer close logic to invoke streams rebalance listener callbacks based on member epoch status
- Refactors background event processor to use the new invoker pattern and removes duplicate callback tracking
Reviewed Changes
Copilot reviewed 6 out of 6 changed files in this pull request and generated 5 comments.
Show a summary per file
File | Description |
---|---|
StreamsRebalanceListenerInvoker.java | New invoker class that handles streams rebalance listener callbacks with logging and error handling |
StreamsRebalanceListenerInvokerTest.java | Comprehensive test coverage for the new invoker class |
AsyncKafkaConsumer.java | Refactored to use the new invoker and updated close logic to trigger streams callbacks |
AsyncKafkaConsumerTest.java | Added tests for the new close behavior with streams listeners |
DefaultStreamsRebalanceListener.java | Added calls to set reconciled assignment after successful task operations |
DefaultStreamsRebalanceListenerTest.java | Updated tests to use mocked StreamsRebalanceData and verify assignment setting |
Tip: Customize your code reviews with copilot-instructions.md. Create the file or learn how to get started.
...t/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java
Outdated
Show resolved
Hide resolved
...t/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java
Show resolved
Hide resolved
...t/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Outdated
Show resolved
Hide resolved
|
||
private Optional<StreamsRebalanceListener> streamsRebalanceListener = Optional.empty(); | ||
private final Optional<StreamsRebalanceData> streamsRebalanceData; | ||
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
out of curiosity: why this is optional, if we anyway will throw exception at line 292 in case of empty optional?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would expect it's optional because it will only exist if the AsycKafkaConsumer is being used with the streams rebalance protocol.
But why do we need to declare it here in the internal class if it already exist in the parent AsyncKafkaConsumer
?
kafka/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Line 345 in 45a410b
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker; |
(can we just use that one? as we do with the
rebalanceListenerInvoker
)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
AsyncConsumer can be used with "plain consumer" or inside "Kafka Streams". We would use this new handler only for the KS case, but not the "plain consumer" case.
And L292 is only executed for the KS case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
True. Removed the second field and used the reference in the outer class instead.
StreamsRebalanceListener firstListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class); | ||
StreamsRebalanceListener secondListener = org.mockito.Mockito.mock(StreamsRebalanceListener.class); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
minor: full import?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
...c/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Made an initial pass. Did not look into the test code yet.
"rebalance protocol events"); | ||
} | ||
this.streamsRebalanceListener = Optional.of(streamsRebalanceListener); | ||
public BackgroundEventProcessor(final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For this constructor, it seems not to make sense to pass in an Optional
? If we are in the KS case, and use this constructor, we must pass the StreamsRebalanceListenerInvoker
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed the constructor.
streamsRebalanceData().setReconciledAssignment(assignment); | ||
} | ||
final Optional<Exception> exceptionFromCallback = Optional.ofNullable(streamsRebalanceListenerInvoker().invokeTasksAssigned(assignment)); | ||
error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task assignment callback throws an error")); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As above (same elsewhere).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
See above.
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Show resolved
Hide resolved
...c/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java
Show resolved
Hide resolved
...c/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvoker.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @lucasbru ! Took a first pass.
|
||
private Optional<StreamsRebalanceListener> streamsRebalanceListener = Optional.empty(); | ||
private final Optional<StreamsRebalanceData> streamsRebalanceData; | ||
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would expect it's optional because it will only exist if the AsycKafkaConsumer is being used with the streams rebalance protocol.
But why do we need to declare it here in the internal class if it already exist in the parent AsyncKafkaConsumer
?
kafka/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Line 345 in 45a410b
private final Optional<StreamsRebalanceListenerInvoker> streamsRebalanceListenerInvoker; |
(can we just use that one? as we do with the
rebalanceListenerInvoker
)
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Show resolved
Hide resolved
if (streamsRebalanceListenerInvoker.isPresent()) { | ||
streamsRebalanceListenerInvoker.get().setRebalanceListener(streamsRebalanceListener); | ||
} else { | ||
throw new IllegalStateException("Consumer was not created to be used with Streams rebalance protocol events"); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would expect is probably best/clearer to do this setup/validation before any actual subscription action (subscribeInternal
)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
|
||
public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment assignment) { | ||
if (listener.isPresent()) { | ||
log.info("Adding newly assigned tasks: {}", assignment); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This log seems a bit confusing here on the callback invocations. Shouldn't it be more along the lines of "trigger task assigned callbacks" (maybe is me misreading, missing KS details. Ex. on the consumer side the assigned callback runs when the partitions were already assigned/added). If this applies, let's align also the logs for revoked/lost
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The consumer rebalance listener logs Adding newly assigned partitions:
in this case, I just copied its behavior for consistency. But generally, I agree.
I updated the log line indicate that the rebalance listener is invoked.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The consumer rebalance listener logs Adding newly assigned partitions: in this case
I see, well, that's wrong too :) at that point the partitions have been already added really. I will fix it on the consumer side ;)
|
||
@Test | ||
public void testCloseInvokesStreamsRebalanceListenerOnAllTasksLostWhenMemberEpochZeroOrNegative() { | ||
// Test that close() calls streamsRebalanceListener.invokeAllTasksLost() when memberEpoch <= 0 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this comment seems to state the same as the func name really, needed?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
assertNotNull(thrownException.getCause()); | ||
assertTrue(thrownException.getCause() instanceof RuntimeException); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
assertNotNull(thrownException.getCause()); | |
assertTrue(thrownException.getCause() instanceof RuntimeException); | |
assertInstanceOf(RuntimeException.class, thrownException.getCause()) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
@Test | ||
public void testInvokeAllTasksRevokedWithNoListener() { | ||
// When no listener is set, should return null | ||
Exception result = invoker.invokeAllTasksRevoked(); | ||
assertNull(result); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
isn't this testing the same as testConstructorInitializesWithEmptyListener
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
...st/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java
Show resolved
Hide resolved
} | ||
|
||
@Test | ||
public void testInvokeAllTasksLostWithNoListener() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
isn't this testing the same as testConstructorInitializesWithEmptyListener
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@lucasbru: Thanks for the patch.
|
||
@Test | ||
public void testSetRebalanceListener() { | ||
// Test setting a listener |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need this comment?
I suppose the test case name is clear enough.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
|
||
@Test | ||
public void testSetRebalanceListenerWithNull() { | ||
// Test setting listener to null |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
...st/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceListenerInvokerTest.java
Show resolved
Hide resolved
Wow, so many reviews. Thanks a lot. I addressed the comments @mjsax @frankvicky @lianetm @UladzislauBlok |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the updates @lucasbru !
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Show resolved
Hide resolved
|
||
public Exception invokeTasksAssigned(final StreamsRebalanceData.Assignment assignment) { | ||
if (listener.isPresent()) { | ||
log.info("Adding newly assigned tasks: {}", assignment); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The consumer rebalance listener logs Adding newly assigned partitions: in this case
I see, well, that's wrong too :) at that point the partitions have been already added really. I will fix it on the consumer side ;)
if (assignedPartitions.isEmpty()) | ||
// Nothing to revoke. | ||
return; | ||
if (streamsRebalanceListenerInvoker.isPresent()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the streamsRebalanceListenerInvoker
could be null here in the case where close
is called from the constructor upon a failure building the consumer (ln 509), so maybe consider adding a check here? (we've been seeing those noisy NPE logs on close recently actually)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Wow, that close is sneaky. Fixed.
applicationEventHandler.add(new CommitOnCloseEvent()); | ||
} | ||
|
||
private void runRebalanceCallbacksOnClose() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
since this func is being extended to cover streams tasks now too, should we tweak the error message to make it more generic? (above, Failed to release group assignment
). Up to you, but maybe something along the lines of "Failed running callbacks" would apply better to both cases
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
public void setRebalanceListener(StreamsRebalanceListener streamsRebalanceListener) { | ||
Objects.requireNonNull(streamsRebalanceListener, "StreamsRebalanceListener cannot be null"); | ||
if (listener.isPresent() && listener.get() != streamsRebalanceListener) { | ||
throw new IllegalStateException("StreamsRebalanceListener can only be set once"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
given this validation, should we clear the listener on unsubscribe
? (otherwise, I expect that subscribe+unsubscribe+subscribe would fail on the last subscribe with this error).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Streams will always use the same rebalance listener to subscribe
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's actually not true - I think I looked at the wrong code. So I changed it make to allow overwriting the rebalance listener on subscribe
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@lianetm Thanks! Addressed / Responded
applicationEventHandler.add(new CommitOnCloseEvent()); | ||
} | ||
|
||
private void runRebalanceCallbacksOnClose() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
if (assignedPartitions.isEmpty()) | ||
// Nothing to revoke. | ||
return; | ||
if (streamsRebalanceListenerInvoker.isPresent()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Wow, that close is sneaky. Fixed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks! LGTM
In the consumer, we invoke the consumer rebalance onPartitionRevoked or onPartitionLost callbacks, when the consumer closes. The point is that the application may want to commit, or wipe the state if we are closing unsuccessfully. In the StreamsRebalanceListener, we did not implement this behavior, which means when closing the consumer we may lose some progress, and in the worst case also miss that we have to wipe our local state state since we got fenced. In this PR we implement StreamsRebalanceListenerInvoker, very similarly to ConsumerRebalanceListenerInvoker and invoke it in Consumer.close.
Co-authored-by: Copilot <[email protected]>
dc8de27
to
3ec8974
Compare
In the consumer, we invoke the consumer rebalance onPartitionRevoked or
onPartitionLost callbacks, when the consumer closes. The point is that
the application may want to commit, or wipe the state if we are closing
unsuccessfully.
In the StreamsRebalanceListener, we did not implement this behavior,
which means when closing the consumer we may lose some progress, and in
the worst case also miss that we have to wipe our local state state
since we got fenced.
In this PR we implement StreamsRebalanceListenerInvoker, very similarly
to ConsumerRebalanceListenerInvoker and invoke it in Consumer.close.
Reviewers: Lianet Magrans [email protected], Matthias J. Sax
[email protected], TengYao Chi [email protected],
Uladzislau Blok [email protected]