-
Notifications
You must be signed in to change notification settings - Fork 14k
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
KAFKA-17154: New consumer subscribe may join group without a call to consumer.poll #17165
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1617,10 +1617,25 @@ public void testRevokePartitionsUsesTopicNamesLocalCacheWhenMetadataNotAvailable | |
} | ||
|
||
@Test | ||
public void testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup() { | ||
public void testOnSubscriptionUpdatedDoesNotTransitionToJoiningIfInGroup() { | ||
ConsumerMembershipManager membershipManager = createMemberInStableState(); | ||
membershipManager.onSubscriptionUpdated(); | ||
assertTrue(membershipManager.subscriptionUpdated()); | ||
membershipManager.onConsumerPoll(); | ||
verify(membershipManager, never()).transitionToJoining(); | ||
assertFalse(membershipManager.subscriptionUpdated()); | ||
} | ||
|
||
@Test | ||
public void testOnSubscriptionUpdatedTransitionsToJoiningOnPollIfNotInGroup() { | ||
ConsumerMembershipManager membershipManager = createMembershipManager(null); | ||
assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); | ||
membershipManager.onSubscriptionUpdated(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should we verify here that there was no There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, added it. Thanks. |
||
verify(membershipManager, never()).transitionToJoining(); | ||
assertTrue(membershipManager.subscriptionUpdated()); | ||
assertEquals(MemberState.UNSUBSCRIBED, membershipManager.state()); | ||
membershipManager.onConsumerPoll(); | ||
verify(membershipManager).transitionToJoining(); | ||
} | ||
|
||
@Test | ||
|
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.
Maybe we need to make
application thread
sleep here to make surenetwork thread
does not send join request.WDYT?
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.
Hi @TaiJuWu,
application
andnetwork
are different threads. I'm not sure why makingapplication
thread sleep can letnetwork
thread doesn't send join request. Also , this test would like to make sureKafkaConsumer#subscribe
doesn't send join request, so we should not have other operations to make the result. Could you elaborate more on your idea? Thank you.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.
Sorry for the confusion.
The reason for making the application sleep here is to give the network enough time to send the join request even without poll. This helps to reproduce the bug more easily and consistently.
Additionally, to ensure the behavior we’ve promised in the documentation, the two operations (subscribe and poll) should not be too close together in the timeline in this test. Does that make sense?
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.
Hi @TaiJuWu, thanks for the suggestion. It's hard to say waiting how much time can make sure
KafkaConsumer#subscribe
doesn't send a join request. I add two unit tests to check the behavior.In
ApplicationEventProcessorTest#testSubscriptionChangeEvent
, we checkApplicationEventProcessor#process(SubscriptionChangeEvent)
doesn't callConsumerMembershipManager#maybeJoinGroup
.In
ConsumerMembershipManagerTest#testOnSubscriptionUpdatedTransitionsToJoiningOnlyIfNotInGroup
, we checkConsumerMembershipManager#onSubscriptionUpdated
keeps member state asUNSUBSCRIBED
and callingtransitionToJoining
inmaybeJoinGroup
.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 your explanation.