You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/11/04 05:24:00 UTC

[jira] [Commented] (KAFKA-6774) Improve default groupId behavior in consumer

    [ https://issues.apache.org/jira/browse/KAFKA-6774?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16674299#comment-16674299 ] 

ASF GitHub Bot commented on KAFKA-6774:
---------------------------------------

vahidhashemian opened a new pull request #5877: KAFKA-6774: Improve the default group id behavior in KafkaConsumer (KIP-289)
URL: https://github.com/apache/kafka/pull/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
   * deprecate the use of empty (`""`) consumer group on the client
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Improve default groupId behavior in consumer
> --------------------------------------------
>
>                 Key: KAFKA-6774
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6774
>             Project: Kafka
>          Issue Type: Improvement
>          Components: consumer
>            Reporter: Jason Gustafson
>            Assignee: Vahid Hashemian
>            Priority: Major
>              Labels: needs-kip
>             Fix For: 2.2.0
>
>
> At the moment, the default groupId in the consumer is "". If you try to use this to subscribe() to a topic, the broker will reject the group as invalid. On the other hand, if you use it with assign(), then the user will be able to fetch and commit offsets using the empty groupId. Probably 99% of the time, this is not what the user expects. Instead you would probably expect that if no groupId is provided, then no committed offsets will be fetched at all and we'll just use the auto reset behavior if we don't have a current position.
> Here are two potential solutions (both requiring a KIP):
> 1. Change the default to null. We will preserve the current behavior for subscribe(). When using assign(), we will not bother fetching committed offsets for the null groupId, and any attempt to commit offsets will raise an error. The user can still use the empty groupId, but they have to specify it explicitly.
> 2. Keep the current default, but change the consumer to treat this value as if it were null as described in option 1. The argument for this behavior is that using the empty groupId to commit offsets is inherently a dangerous practice and should not be permitted. We'd have to convince ourselves that we're fine not needing to allow the empty groupId for backwards compatibility though.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)