Skip to content

Conversation

@ShivsundarR
Copy link
Contributor

@ShivsundarR ShivsundarR commented Jan 2, 2026

What

  • Currently if a consumer/share-consumer calls close() before it has
    joined a group, then the heartbeat on close will be sent with epoch =
    -1 and the broker would return "GroupIdNotFoundException".
  • This was causing couple of tests in ShareConsumerTest to be flaky if
    the heartbeat to join the group was sent with epoch = -1.
  • Since this can occur in real scenarios as well, it would be better to
    tolerate this exception while we are leaving the group so that the
    consumer can close cleanly.

Reviewers: Andrew Schofield [email protected]

@github-actions github-actions bot added triage PRs from the community consumer clients labels Jan 2, 2026
@AndrewJSchofield AndrewJSchofield added KIP-932 Queues for Kafka ci-approved and removed triage PRs from the community labels Jan 2, 2026
@AndrewJSchofield AndrewJSchofield self-requested a review January 2, 2026 14:46
@AndrewJSchofield
Copy link
Member

@ShivsundarR I still get a failure of testSubscribeOnInvalidTopicThrowsInvalidTopicException even with this patch.

@ShivsundarR
Copy link
Contributor Author

Thanks @AndrewJSchofield, I was using the incorrect state for the check, it should actually be MemberState.UNSUBSCRIBED, since we move from MemberState.LEAVING to MemberState.UNSUBSCRIBED after sending a leave request.
I have made this change now, and ran the test locally for 100 times and did not get a flake.

Copy link
Member

@AndrewJSchofield AndrewJSchofield left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the PR. I think the revised member state in the check is now correct. A few very minor comments only.

heartbeatRequestName(), error, errorMessage);
handleFatalFailure(error.exception(errorMessage));
}
break;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Let's have a blank line following the break to match the other cases in this switch.

membershipManager().onHeartbeatRequestSkipped();
} else {
// Else, this is a fatal error, we should throw it and transition to fatal state.
logger.error("{} failed due to unexpected error {}: {}",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: The following logger.error is identical but on a single line. Please change this to a single line also.

// from LEAVING to UNSUBSCRIBED in onHeartbeatRequestGenerated() before the request is sent.
if (membershipManager().state() == MemberState.UNSUBSCRIBED) {
logger.info("{} received GROUP_ID_NOT_FOUND for group {} while unsubscribed. " +
"Not treating as fatal since consumer is leaving group.",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would remove the "Not treating as fatal since consumer is leaving group.". There's no sense in using "fatal" in an information log line. This is nothing to worry about in the slightest.

@AndrewJSchofield AndrewJSchofield self-requested a review January 6, 2026 09:18
@AndrewJSchofield AndrewJSchofield merged commit aee7a37 into apache:trunk Jan 6, 2026
21 checks passed
Parkerhiphop pushed a commit to Parkerhiphop/kafka that referenced this pull request Jan 6, 2026
…group. (apache#21239)

*What*

- Currently if a consumer/share-consumer calls `close()` before it has
joined a group, then the heartbeat on close will be sent with `epoch` =
-1 and the broker would return "`GroupIdNotFoundException`".
- This was causing couple of tests in `ShareConsumerTest` to be flaky if
the heartbeat to join the group was sent with `epoch` = -1.
- Since this can occur in real scenarios as well, it would be better to
tolerate this exception while we are leaving the group so that the
consumer can close cleanly.

Reviewers: Andrew Schofield <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants