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-13891: reset generation when syncgroup failed with REBALANCE_IN_PROGRESS #12140
KAFKA-13891: reset generation when syncgroup failed with REBALANCE_IN_PROGRESS #12140
Conversation
Thanks for the patch. Could we file a Jira for it please? |
created a jira for it: https://issues.apache.org/jira/browse/KAFKA-13891 |
@showuon can you please help to review this? |
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.
@aiquestion , thanks for the PR. Could we add a test for it?
@@ -807,6 +807,7 @@ public void handle(SyncGroupResponse syncResponse, | |||
} else if (error == Errors.REBALANCE_IN_PROGRESS) { | |||
log.info("SyncGroup failed: The group began another rebalance. Need to re-join the group. " + | |||
"Sent generation was {}", sentGeneration); | |||
resetStateOnResponseError(ApiKeys.SYNC_GROUP, error, false); |
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.
We might need to add a comment here to explain why we need to reset generation ID here.
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.
added comment & unit test. thanks~
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.
LGTM! Thanks for the update. Left a minor comment.
AbstractCoordinator.Generation currentGeneration = coordinator.generation(); | ||
return currentGeneration.generationId == AbstractCoordinator.Generation.NO_GENERATION.generationId && | ||
currentGeneration.memberId.equals(memberId); | ||
}, 2000, "Generation should be reset"); |
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.
nit: I saw the 2000
timeout appeared in AbstractCoordinatorTest.java
in many places. Could we use a static variable to replace them? Thanks.
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.
can't think of any name for this 2000 timeout. so i just changed it to rebalance timeout. does that make sense? :-p
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.
@aiquestion , the current REBALANCE_TIMEOUT_MS
is 60 seconds, which means we'll wait 60 secs for generation reset. It's not correct. It should use 2 seconds as before. I think you can ignore my previous minor comment about 2000
change, and revert the REBALANCE_TIMEOUT_MS
back to 2000
. 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.
ah, okay. reverted. Thanks~
@showuon sorry for the delay. What should i do to get this PR merged? ( it's my first time submit a PR ) |
050c1ce
to
8d425b0
Compare
Failed tests are unrelated
|
CONFLUENT: Sync from apache/kafka trunk to confluentinc/kafka master (13 Jun 2022) apache/trunk: (7 commits) KAFKA-13891: reset generation when syncgroup failed with REBALANCE_IN…(apache#12140) KAFKA-10000: Exactly-once source tasks (apache#11780) KAFKA-13436: Omitted BrokerTopicMetrics metrics in the documentation (apache#11473) MINOR: Use Exit.addShutdownHook instead of directly adding hooks to R…(apache#12283) KAFKA-13846: Adding overloaded metricOrElseCreate method (apache#12121) KAFKA-13935 Fix static usages of IBP in KRaft mode (apache#12250) HOTFIX: null check keys of ProducerRecord when computing sizeInBytes (apache#12288) Conflicts: None
…LANCE_IN_PROGRESS (apache#12140)" This reverts commit c23d60d.
Hey @dajac @showuon just came across this from a user who's running into this on 3.0, given it was part of a series of fixes leading up to/included in 3.0, I think it can/should be backported to 3.2 - 3.0. Any concerns there? Just lmk if there's any reason to be careful, or changes needed to backport the fix faithfully |
…_PROGRESS (#12140) Reviewers: Luke Chen <showuon@gmail.com>
Agree to backported to 3.2 - 3.0. Thanks. |
Thanks @showuon ! Unfortunately I'm now seeing that the situation may be more complicated than I'd initially thought :/ Just came across this followup to the patch here: https://issues.apache.org/jira/browse/KAFKA-14016 Original reporter/PR author actually suggests reverting the changes here, and offers an alternative fix. I feel like I'm still catching up on the whole history here but while I wrap my head around could you give this new ticket a look? Wondering what your take on this is Don't want to bias you with this but FWIW, when I was first pointed to this PR I was definitely skeptical of the changes, though I did ultimately convince myself it made sense. Now I'm letting my doubts creep back in lol |
…LANCE_IN_PROGRESS (apache#12140)" This reverts commit c23d60d.
…LANCE_IN_PROGRESS (apache#12140)" (apache#12794) This reverts commit c23d60d. Reviewers: Luke Chen <showuon@gmail.com>
This PR is a missing part of #11451
Previous change want to solve https://issues.apache.org/jira/browse/KAFKA-13419, but in the final code didn't add code to reset generation id when SyncGroup received REBALANCE_IN_PROGRESS error.
Committer Checklist (excluded from commit message)