chia7712 commented on code in PR #17418:
URL: https://github.com/apache/kafka/pull/17418#discussion_r1804514266
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -358,12 +358,19 @@ private void process(final ShareAcknowledgeAsyncEvent
event) {
* consumer join the share group if it is not part of it yet, or send the
updated subscription if
* it is already a member.
*/
- private void process(final ShareSubscriptionChangeEvent ignored) {
+ private void process(final ShareSubscriptionChangeEvent event) {
if (!requestManagers.shareHeartbeatRequestManager.isPresent()) {
- log.warn("Group membership manager not present when processing a
subscribe event");
+ KafkaException error = new KafkaException("Group membership
manager not present when processing a subscribe event");
+ event.future().completeExceptionally(error);
return;
}
+
+ if (subscriptions.subscribeToShareGroup(event.topics()))
+ metadata.requestUpdateForNewTopics();
Review Comment:
It seems this behavior is aligned with the async consumer, so my comment is
not meant to block this PR. However, I'd like to discuss it for my own
understanding 😄.
We only request to update the metadata cache here after the subscription
changes. However, #17165 adds the condition if
(subscriptionUpdated.compareAndSet(true, false) && state ==
MemberState.UNSUBSCRIBED), which prevents the consumer from rejoining the group
if it’s already in the group.
This means the consumer could send a request with out-of-date assignments
(since it doesn’t rejoin the group) but with updated metadata (since it
subscribes to new topics). For example:
```java
consumer.subscribe(List.of("topic_a"));
...
consumer.subscribe(List.of("topic_b"));
```
If the topics in the second subscription weren't included in the previous
subscription, this could cause a FETCH_SESSION_TOPIC_ID_ERROR until the
consumer rejoins the group via a heartbeat.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]