Skip to content

KAFKA-3854: Fix issues with new consumer's subsequent regex (pattern) subscriptions#1572

Closed
vahidhashemian wants to merge 1 commit intoapache:trunkfrom
vahidhashemian:KAFKA-3854
Closed

KAFKA-3854: Fix issues with new consumer's subsequent regex (pattern) subscriptions#1572
vahidhashemian wants to merge 1 commit intoapache:trunkfrom
vahidhashemian:KAFKA-3854

Conversation

@vahidhashemian
Copy link
Contributor

@vahidhashemian vahidhashemian commented Jun 29, 2016

This patch fixes two issues:

  1. Subsequent regex subscriptions fail with the new consumer.
  2. Subsequent regex subscriptions would not immediately refresh metadata to change the subscription of the new consumer and trigger a rebalance.

The final note on the JIRA stating that a later created topic that matches a consumer's subscription pattern would not be assigned to the consumer upon creation seems to be as designed. A repeat
subscribe() to the same pattern or some wait time until the next automatic metadata refresh would handle that case.

An integration test was also added to verify these issues are fixed with this PR.

Copy link
Contributor

Choose a reason for hiding this comment

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

Nitpick: the use of 'confirm' in the name makes it sound like this is just a check, but it actually mutates internal state. Maybe setSubscriptionType would be a little more accurate?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I was having some doubts when picking this name. The method both sets the state (if not set yet) and verifies it (if set). I'll change it to setSubscriptionType. Thanks.

@vahidhashemian vahidhashemian force-pushed the KAFKA-3854 branch 2 times, most recently from 7503abb to 7446115 Compare June 30, 2016 21:10
@hachikuji
Copy link
Contributor

@vahidhashemian Thanks for the patch! Would it be much trouble to add an integration test (e.g. in PlaintextConsumerTest to verify that a second call to subscribe(Pattern) works correctly?

@vahidhashemian
Copy link
Contributor Author

@hachikuji Sure. I can do that. I had opened KAFKA-3897 to improve regex subscription testing in a separate PR.

@hachikuji
Copy link
Contributor

That works for me. Another small thing: could we change the title of the PR to reflect the problem you found (i.e. metadata should be refreshed immediately upon changing the subscription regex).

@vahidhashemian
Copy link
Contributor Author

Will do that too. Thanks.

@vahidhashemian
Copy link
Contributor Author

vahidhashemian commented Jun 30, 2016

@hachikuji Actually, the PR fixes two issues. Metadata refresh fixes one (item 2 in description), and the SubscriptionType added fixes the other (item 1 in description). Do you think the title you suggested still works?

@hachikuji
Copy link
Contributor

Good point. In that case, the current title probably works.

@vahidhashemian
Copy link
Contributor Author

@hachikuji I added an integration test that covers both reported issues. Please advise if you see an issue. Thanks.

@vahidhashemian
Copy link
Contributor Author

vahidhashemian commented Jul 1, 2016

Weird, some tests are randomly failing (first, second, last try), and all of the failures are from TestUtils.waitUntilTrue(...). Perhaps the default 5000 ms is not a long enough wait time? I increased the wait time on the pattern subscription tests on the third try and they all passed. None fails when I test locally.

I see some other builds, like this and this, also failed similarly.

@hachikuji
Copy link
Contributor

@vahidhashemian LGTM overall. I ran the test a few times locally and couldn't reproduce the failure, but jenkins tends to be more erratic. The only thought that comes to mind is that we somehow don't have up-to-date metadata when the consumer joins the group the first time.

@vahidhashemian
Copy link
Contributor Author

@hachikuji Could you please elaborate? Do you see that as a side-effect of this update, or something that still hasn't been addressed? Thanks.

@hachikuji
Copy link
Contributor

@vahidhashemian I don't see an actual problem, I'm just speculating what could cause the test to timeout. If you can reproduce the failure and see the consumer logs, we should be able to tell what's going on.

@vahidhashemian
Copy link
Contributor Author

vahidhashemian commented Jul 1, 2016

Aah! Thanks for clarifying. I'll take a look at the logs. One thing I noticed happening occasionally for me locally, when I was testing my sample Java consumer, was a delay of over 20 seconds between when onPartitionsRevoked() and onPartitionsAssigned() were called. Not sure if that's related or not.

Copy link
Contributor

Choose a reason for hiding this comment

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

@vahidhashemian The patch looks good, I just want to call out one thing that this test doesn't actually cover. We have other tests that validate that when we add topics, after a metadata refresh the new topic is included in the subscription when it matches the regex. This test, by checking for the removal of some topics, already validates that when metadata is updated we'll see the subscription updated to reflect the new regex. However, the cause of the metadata refresh may be either a) the resubscription or b) the normal metadata refresh interval. For (b), to make some other tests fast, the base class sets it to only 100ms. If we want this test to also validate that the resubscription forces an immediate metadata refresh and the subscription change is reflected promptly, I think we'd need to use a consumer with the metadata refresh interval overridden (see a few other tests in here that use custom settings for an example), add another topic before the resubscription (the only thing that will validate metadata changes), make sure it is included in the new regex, and then validate as we are already.

I think this is just some minor tweaking of the test, so hopefully straightforward to validate. A comment explaining what we're trying to make sure is happening would also be helpful as @hachikuji and I just had to spend some time thinking through what one of the other tests was actually evaluating.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@ewencp Thanks for your feedback and catching the missing test. I'll add the test and the comments as you suggested.

@vahidhashemian
Copy link
Contributor Author

@ewencp I updated the test to cover what you mentioned. Also added some comments for the three pattern subscription tests. I realized that unsubscription is actually being tested in all three, so perhaps testPatternUnsubscription() is redundant.

@vahidhashemian vahidhashemian reopened this Jul 2, 2016
… subscriptions

This patch fixes two issues:
1. Subsequent regex subscriptions fail with the new consumer.
2. Subsequent regex subscriptions would not actually refresh metadata and change the subscription of the new consumer nor they would trigger a rebalance.

The final note on the JIRA stating that a later created topic that matches a consumer's subscription pattern would not be assigned to the consumer upon creation seems to be as designed. A repeat subscribe() to the same pattern would be needed to handle that case.
Unit tests for regex subscriptions will be handled in KAFKA-3897.
@ewencp
Copy link
Contributor

ewencp commented Jul 4, 2016

LGTM, thanks @vahidhashemian for working through all the comments and improving docs on the existing tests!

@asfgit asfgit closed this in d7de59a Jul 4, 2016
@ijuma
Copy link
Member

ijuma commented Jul 4, 2016

@ewencp, seems like we should double-commit this to 0.10.0. What do you think?

@ewencp
Copy link
Contributor

ewencp commented Jul 4, 2016

@ijuma I'm open to it, but skeptical -- would you consider this a critical fix that must go into 0.10.0.1? It's an unfortunate misbehavior, but seems like an edge case I wouldn't consider critical. Updating regex subscriptions dynamically is not a particularly broad use case and this was only found through testing of a KIP as I understand it, not someone hitting a production issue with the new consumer.

All that said, if you want to cherry-pick, I will not object :)

@ijuma
Copy link
Member

ijuma commented Jul 4, 2016

@ewencp It's probably not critical, but it's a clear bug fix, it seemed low risk and it could surprise users in production. I wouldn't call an edge case because it's so easy to trigger (it's not like it requires a complex sequence of operations) even if it may be a bit rare in practice.

Do you agree that it's low risk? Happy to cherry-pick, if so. Otherwise, better to leave it indeed.

asfgit pushed a commit that referenced this pull request Jul 4, 2016
… subscriptions

This patch fixes two issues:
1. Subsequent regex subscriptions fail with the new consumer.
2. Subsequent regex subscriptions would not immediately refresh metadata to change the subscription of the new consumer and trigger a rebalance.

The final note on the JIRA stating that a later created topic that matches a consumer's subscription pattern would not be assigned to the consumer upon creation seems to be as designed. A repeat
`subscribe()` to the same pattern or some wait time until the next automatic metadata refresh would handle that case.

An integration test was also added to verify these issues are fixed with this PR.

Author: Vahid Hashemian <vahidhashemian@us.ibm.com>

Reviewers: Jason Gustafson <jason@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #1572 from vahidhashemian/KAFKA-3854
@ewencp
Copy link
Contributor

ewencp commented Jul 4, 2016

@ijuma Pushed to 0.10.0, and updated JIRA fix version. We should probably figure out how to distinguish between critical, low-risk, and not, but in this case, agreed I'm not too worried about existing functionality or compatibility.

@ijuma
Copy link
Member

ijuma commented Jul 4, 2016

Thanks!

@reftel
Copy link
Contributor

reftel commented Jul 5, 2016

PlaintextConsumerTest.testPatternUnsubscription and PlaintextConsumerTest.testSubsequentPatternSubscription fail for me on commit 87b3ce with JDK 1.8.0_31. The failure messages are:

java.lang.AssertionError: Expected partitions [topic-0, topic-1, tblablac-0, tblablac-1] but actually got []
and

java.lang.AssertionError: Expected partitions [topic-0, topic-1, foo-0] but actually got []

All other test cases pass.

@vahidhashemian
Copy link
Contributor Author

@MagnusR Looking at the recent failed builds it seems the issue is intermittent where one or more pattern subscription tests are failing, but there is no one test that always fail. I assume all tests pass locally for you?

@ewencp
Copy link
Contributor

ewencp commented Jul 5, 2016

@MagnusR Yeah, would be good to know if this is consistent and only this test fails. Because we basically have system integration tests as unit tests, they end up causing transient failures. We've noticed some other tests failing under Jenkins and many seem to be relying on the default timeout from TestUtils.waitUntilTrue, so we may just need to increase that to make it more reliable across a variety of test hardware.

@reftel
Copy link
Contributor

reftel commented Jul 5, 2016

@vahidhashemian: This was a local run on my laptop. 804 test run, 2 tests failed. Only tried once, so I don't know if it's intermittent or not.

granthenke pushed a commit to granthenke/kafka that referenced this pull request Oct 24, 2016
… subscriptions

This patch fixes two issues:
1. Subsequent regex subscriptions fail with the new consumer.
2. Subsequent regex subscriptions would not immediately refresh metadata to change the subscription of the new consumer and trigger a rebalance.

The final note on the JIRA stating that a later created topic that matches a consumer's subscription pattern would not be assigned to the consumer upon creation seems to be as designed. A repeat
`subscribe()` to the same pattern or some wait time until the next automatic metadata refresh would handle that case.

An integration test was also added to verify these issues are fixed with this PR.

Author: Vahid Hashemian <vahidhashemian@us.ibm.com>

Reviewers: Jason Gustafson <jason@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes apache#1572 from vahidhashemian/KAFKA-3854
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants