KAFKA-6774: Improve the default group id behavior in KafkaConsumer (KIP-289)#5877
KAFKA-6774: Improve the default group id behavior in KafkaConsumer (KIP-289)#5877hachikuji merged 9 commits intoapache:trunkfrom
Conversation
0b0dec0 to
73c9ba1
Compare
hachikuji
left a comment
There was a problem hiding this comment.
@vahidhashemian Thanks for the PR. I think we also might need to hook into the logic for setting the initial position of the consumer. By default, we attempt to fetch committed offsets, but if the group id is null, we should just skip this step.
There was a problem hiding this comment.
Perhaps group management or the the offset commit api?
73c9ba1 to
3e4b22c
Compare
|
@hachikuji Thanks for reviewing the PR. Submitted a patch to address your feedback. |
3e4b22c to
f775ccb
Compare
hachikuji
left a comment
There was a problem hiding this comment.
Left a couple more comments. Do we have an integration test that covers basic fetching using a null group id? We should probably make sure we have at least one case which uses null and one which uses the empty string.
There was a problem hiding this comment.
I think it's fine to change the default as long as the user hasn't explicitly provided the configuration. However, if they have explicitly set enable.auto.commit=true, then it might be better to raise an exception.
There was a problem hiding this comment.
We should probably have a fail below if this returns without errors?
There was a problem hiding this comment.
Thanks! Totally missed them.
vahidhashemian
left a comment
There was a problem hiding this comment.
@hachikuji I tried to address your recent comments in the new commit. Thanks for the additional feedback.
There was a problem hiding this comment.
Thanks! Totally missed them.
hachikuji
left a comment
There was a problem hiding this comment.
Thanks for the tests. Left a few minor comments.
There was a problem hiding this comment.
Is it possible to just pass "groupId=null" as an override?
There was a problem hiding this comment.
Doesn't seem so, I get a NPE:
java.lang.NullPointerException
at java.base/java.util.concurrent.ConcurrentHashMap.putVal(ConcurrentHashMap.java:1021)
at java.base/java.util.concurrent.ConcurrentHashMap.put(ConcurrentHashMap.java:1016)
at java.base/java.util.Properties.put(Properties.java:1309)
at kafka.api.PlaintextConsumerTest.testConsumingWithNullGroupId(PlaintextConsumerTest.scala:1838)
...
There was a problem hiding this comment.
I suppose we could leave this out?
There was a problem hiding this comment.
That's correct.
There was a problem hiding this comment.
Seems more accurate to say that consumer 3 begins from an explicit seek?
There was a problem hiding this comment.
nit: maybe better to turn this into explicit equals assertions so that we know which check failed.
There was a problem hiding this comment.
Sounds good.
There was a problem hiding this comment.
The third iteration here feels redundant. Doesn't it just read from the committed offset like consumer2?
There was a problem hiding this comment.
Yes, I put it to verify the auto.offset.reset doesn't affect the behavior. I'll remove it in the new commit.
vahidhashemian
left a comment
There was a problem hiding this comment.
@hachikuji I updated the unit tests based on your comments. Thanks!
There was a problem hiding this comment.
That's correct.
There was a problem hiding this comment.
Doesn't seem so, I get a NPE:
java.lang.NullPointerException
at java.base/java.util.concurrent.ConcurrentHashMap.putVal(ConcurrentHashMap.java:1021)
at java.base/java.util.concurrent.ConcurrentHashMap.put(ConcurrentHashMap.java:1016)
at java.base/java.util.Properties.put(Properties.java:1309)
at kafka.api.PlaintextConsumerTest.testConsumingWithNullGroupId(PlaintextConsumerTest.scala:1838)
...
There was a problem hiding this comment.
Yes, I put it to verify the auto.offset.reset doesn't affect the behavior. I'll remove it in the new commit.
There was a problem hiding this comment.
Sounds good.
hachikuji
left a comment
There was a problem hiding this comment.
@vahidhashemian Thanks for the patch. Looks good overall. I had just one thought. When the groupId is null, there should be no coordinator interaction at all. We could make this explicit by not bothering to construct the ConsumerCoordinator object. That would take it out of the poll path and reduce the risk of using it accidentally. What do you think?
|
@hachikuji thanks for the feedback. Your suggestion makes sense. Even though it's not explicitly mentioned in the KIP, I think it wouldn't result in a different behavior that what is covered by the KIP. I submitted a patch that should cover it (it did not break any test locally), but please let me know if I missed something. Thanks! |
e5b9430 to
db68a67
Compare
hachikuji
left a comment
There was a problem hiding this comment.
Thanks for the updates, just a few additional comments.
There was a problem hiding this comment.
Perhaps now we don't need this? We could add an assertion in the constructor that the groupId is not null.
There was a problem hiding this comment.
You're right. I added an assertion in AbstractCoordinator constructor. It seemed to be a better fit there.
There was a problem hiding this comment.
nit: not sure if there's a strong reason for any of these to use the empty group id. Could we use groupId instead?
There was a problem hiding this comment.
No particular reason other than I wanted to stick to the actual group id that was used before. I updated these to use the provided groupId.
There was a problem hiding this comment.
I was wondering if we could rephrase this to suggest what the user should do more clearly. For example, "To use the group management or offset commit APIs, you must provide a valid group.id in the consumer configuration."
There was a problem hiding this comment.
Sounds good, thanks!
There was a problem hiding this comment.
Just to complete the integration picture, could we make a call to committed and assert the exception?
There was a problem hiding this comment.
I did that in the next commit.
vahidhashemian
left a comment
There was a problem hiding this comment.
@hachikuji thanks for the feedback/suggestions. The new commit should address them.
There was a problem hiding this comment.
You're right. I added an assertion in AbstractCoordinator constructor. It seemed to be a better fit there.
There was a problem hiding this comment.
Sounds good, thanks!
There was a problem hiding this comment.
No particular reason other than I wanted to stick to the actual group id that was used before. I updated these to use the provided groupId.
There was a problem hiding this comment.
I did that in the next commit.
|
@vahidhashemian Can you look into the build failures? I looked at one of the streams tests ( |
|
Thanks @hachikuji for catching this. which now means setting the group id to null. I think it should be safe to replace it with Thoughts? |
|
The exception is being raised from a call to |
|
Sounds good, and seems to have no side effect with respect to existing unit tests. My initial thought was to use the empty string to keep the tests run with the same config as before. I'll submit a patch. |
f6fec6c to
05a7594
Compare
hachikuji
left a comment
There was a problem hiding this comment.
Thanks for the updates. Looks like the build is passing. Just two more comments and I think we can merge.
There was a problem hiding this comment.
Shouldn't we handle the else case? I was thinking that if the user enables auto commit explicitly, but provides no group id, then we should raise a configuration exception.
There was a problem hiding this comment.
What I had here was to let the consumer run and throw an exception when a commit is triggered (or even earlier when e.g. subscribe is called). But I think your suggestion makes more sense because it points directly to the root cause, i.e. the misconfiguration. Updated in the new patch.
There was a problem hiding this comment.
nit: can we use Objects.requireNonNull instead?
There was a problem hiding this comment.
Made this change too.
…IP-289) Improve the default group id behavior by * changing the default consumer group to null, where no offset commit or fetch, or group management operations are allowed * deprecate the use of empty (`""`) consumer group on the client
Also modified the logic in deciding whether auto-commit should be enable.
The validation caused Streams tests in `KTableSourceTopicRestartIntegrationTest` to fail.
05a7594 to
7f5941e
Compare
vahidhashemian
left a comment
There was a problem hiding this comment.
@hachikuji Thanks for the feedback. Responded to your comment inline and submitted a new patch.
There was a problem hiding this comment.
What I had here was to let the consumer run and throw an exception when a commit is triggered (or even earlier when e.g. subscribe is called). But I think your suggestion makes more sense because it points directly to the root cause, i.e. the misconfiguration. Updated in the new patch.
There was a problem hiding this comment.
Made this change too.
hachikuji
left a comment
There was a problem hiding this comment.
LGTM. Pushed a trivial fix for the null check in AbstractCoordinator.
…IP-289) (apache#5877) Improve the default group id behavior by: * changing the default consumer group to null, where no offset commit or fetch, or group management operations are allowed * deprecating the use of empty (`""`) consumer group on the client Reviewers: Jason Gustafson <jason@confluent.io>
Improve the default group id behavior by
"") consumer group on the clientCommitter Checklist (excluded from commit message)