Skip to content
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

Merged
merged 2 commits into from Jun 13, 2022

Conversation

aiquestion
Copy link
Contributor

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)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@dajac
Copy link
Contributor

dajac commented May 9, 2022

Thanks for the patch. Could we file a Jira for it please?

@aiquestion aiquestion marked this pull request as ready for review May 10, 2022 01:28
@aiquestion aiquestion changed the title MINOR: reset generation when syncgroup failed with REBALANCE_IN_PROGRESS KAFKA-13891: reset generation when syncgroup failed with REBALANCE_IN_PROGRESS May 10, 2022
@aiquestion
Copy link
Contributor Author

created a jira for it: https://issues.apache.org/jira/browse/KAFKA-13891

@aiquestion
Copy link
Contributor Author

@showuon can you please help to review this?

Copy link
Contributor

@showuon showuon left a 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);
Copy link
Contributor

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.

Copy link
Contributor Author

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~

Copy link
Contributor

@showuon showuon left a 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");
Copy link
Contributor

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.

Copy link
Contributor Author

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

Copy link
Contributor

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ah, okay. reverted. Thanks~

@aiquestion
Copy link
Contributor Author

@aiquestion , thanks for the PR. Could we add a test for it?

@showuon sorry for the delay. What should i do to get this PR merged? ( it's my first time submit a PR )

@aiquestion aiquestion force-pushed the reset_generation_syncgroup_fail branch from 050c1ce to 8d425b0 Compare June 12, 2022 10:35
@showuon
Copy link
Contributor

showuon commented Jun 13, 2022

Failed tests are unrelated

    Build / JDK 11 and Scala 2.13 / kafka.server.MultipleListenersWithDefaultJaasContextTest.testProduceConsume()
    Build / JDK 11 and Scala 2.13 / kafka.server.UpdateFeaturesTest.testShouldFailRequestDuringDeletionOfNonExistingFeature()

@showuon showuon merged commit c23d60d into apache:trunk Jun 13, 2022
@aiquestion aiquestion deleted the reset_generation_syncgroup_fail branch June 13, 2022 03:21
ableegoldman added a commit to confluentinc/kafka that referenced this pull request Jun 14, 2022
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
aiquestion added a commit to aiquestion/kafka that referenced this pull request Jun 25, 2022
@ableegoldman
Copy link
Contributor

ableegoldman commented Oct 27, 2022

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

ableegoldman pushed a commit that referenced this pull request Oct 27, 2022
…_PROGRESS (#12140)

Reviewers: Luke Chen <showuon@gmail.com>
@showuon
Copy link
Contributor

showuon commented Oct 27, 2022

Agree to backported to 3.2 - 3.0. Thanks.

@ableegoldman
Copy link
Contributor

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

aiquestion added a commit to aiquestion/kafka that referenced this pull request Oct 28, 2022
showuon pushed a commit that referenced this pull request Nov 5, 2022
…LANCE_IN_PROGRESS (#12140)" (#12794)

This reverts commit c23d60d.

Reviewers: Luke Chen <showuon@gmail.com>
showuon pushed a commit that referenced this pull request Nov 5, 2022
…LANCE_IN_PROGRESS (#12140)" (#12794)

This reverts commit c23d60d.

Reviewers: Luke Chen <showuon@gmail.com>
showuon pushed a commit that referenced this pull request Nov 5, 2022
…LANCE_IN_PROGRESS (#12140)" (#12794)

This reverts commit c23d60d.

Reviewers: Luke Chen <showuon@gmail.com>
guozhangwang pushed a commit to guozhangwang/kafka that referenced this pull request Jan 25, 2023
…LANCE_IN_PROGRESS (apache#12140)" (apache#12794)

This reverts commit c23d60d.

Reviewers: Luke Chen <showuon@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants