Status

Current state: Accepted

Discussion thread: here

Vote thread: here

JIRA: Unable to render Jira issues macro, execution error.

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

As explained in the corresponding JIRA, the consumer behavior is not consistent when it comes to using an empty ("") group id (which is the default):

  • If the consumer subscribes to a topic using this group id it will get an invalid group id error from the broker during a poll (JoinGroupRequest fails group validation if group id is ""):
    Exception in thread "main" org.apache.kafka.common.errors.InvalidGroupIdException: The configured groupId is invalid
  • For a stand-alone consumer, that assigns partitions to itself, using the default group id ("") does not cause an error. It can fetch last committed offset, seek to a partition offset, fetch messages, and commit offsets without having to worry about the empty string group id.

This has been summarized in table below.


When using subscribe(...)
(group management)
When using assign(...)
(standalone consumer)
group.id=""InvalidGroupException
  • Can fetch (last committed) offset
  • Can seek to specific offset
  • Can fetch from specific offset
  • Can commit offset

This can be confusing to users as they would likely expect the consumer treat the default group id the same way in these two cases. Using the default group id of "" is troublesome, and consumers are not normally expected to do that in a production setting. This KIP proposes deprecating the empty group id on the client, and changing the default group id to null so that unnecessary and unintentional offset fetches and commits are avoided.

Public Interfaces

  • The default group id will change from "" to null so that consumers that want to use an empty ("") group id would have to explicitly specify it.
  • The use of empty group id will be deprecated on the client, and consumers using this group id will receive a warning about this deprecation. Client support for empty group id will be removed in the next major release. 

Proposed Changes

Change Default group.id to null

The default value of consumer config group.id will be changed from "" to null. This means that if standalone consumers want to keep using the group id "" they would have to explicitly provide the config (e.g. props.put(ConsumerConfig.GROUP_ID_CONFIG, "");). The behavior would remain the same otherwise.

This is how these two group ids will work


When using subscribe(...)
(group management)

When using assign(...)
(standalone consumer)

group.id=null (new default)

InvalidGroupException
  • Cannot fetch offsets (uses auto.offset.reset config if no current offset is available)
  • Cannot commit offsets (enable.auto.commit is automatically set to false)

group.id="" (specified by user)

InvalidGroupException
  • Can fetch (last committed) offset
  • Can seek to specific offset
  • Can fetch from specific offset
  • Can commit offset


Client Side Deprecation

The use of empty group id will be deprecated on the client. A warning log will appear on clients that use this group id. 

Compatibility, Deprecation, and Migration Plan

The default group id will change from "" to null. Therefore, users who rely on "" as the default group id will have to explicitly specify it. Otherwise, null will be picked for group id, which does not support offset fetch/commit. This may still be OK for those who do not rely on offset fetch/commit when using the current default.

Rejected Alternatives

  • Keeping "" as the default group id: The reason for not choosing this option is that using the empty group id could be unknowingly done at the moment (because of it being the default), and since it is not a recommended practice and could lead to issues, it will no longer be the default. Anyone who wants to keep using it should explicitly specify it and be aware if the restrictions set around it by this KIP.
  • Deprecating and dropping support for the use of empty group id on the broker: This was also ruled out to support backward compatibility and allow consumers that rely on the existing broker behavior with respect to this group id to be able to do so in the future.
  • No labels