You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 10 Next »

Status

Current state: Under Discussion

Discussion 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 immediately, and dropping its support by the next major release. It also suggests changing the default group id to help with this phase-out plan.

Public Interfaces

  • The default group id will also change to null so that consumers that want to use an empty ("") group id for coordinator-free consumption would have to explicitly specify that group id.
  • A new version of OffsetCommit API will be created that does not support the empty group id. The clients will fall back to older version of this API in case the user explicitly chooses to use the empty group id.

Proposed Changes

This is a detailed list of improvements proposed by this KIP:

  1. Changing the default group id to null 
  2. A new OffsetCommit API version in which, the group id "" is not supported.
  3. The clients will use (until the next major release) the older version of the API in case user explicitly specifies the group id as ""

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, "");).

This is how these two group ids will work


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

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

group.id=null (default)

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

group.id="" (specified by user)

InvalidGroupException (like before)
  • Until the next major release: Works like before (clients use the older version of OffsetCommit)
  • At the next major release: Results in INVALID_GROUP_ID error

OffsetCommit API Enhancement

The current OffsetCommit protocol looks like this.

OffsetCommit Protocol - Version 5
OffsetCommit Request (Version: 5) => group_id generation_id member_id retention_time [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time => INT64
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING


OffsetCommit Response (Version: 5) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16

The new version of the protocol is the same, with an additional constraint that committing offsets when group_id is "", results in INVALID_GROUP_ID.

OffsetCommit Protocol - Version 6
OffsetCommit Request (Version: 6) => group_id generation_id member_id retention_time [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time => INT64
  topics => topic [partitions] 
    topic => STRING
    partitions => partition offset metadata 
      partition => INT32
      offset => INT64
      metadata => NULLABLE_STRING


OffsetCommit Response (Version: 6) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16

Client Side Changes

If users explicitly use "" as the group id the client will switch to using version 5 of the OffsetCommit API for offset commits. For all other group ids version 6 will be used.

After the next major release the requests with an empty group id will also be handled by version 6 of the API, which results in an INVALID_GROUP_ID error.

Compatibility, Deprecation, and Migration Plan

The empty group id will be deprecated and its support will be removed in the next major release. For now, a warning will be logged to indicate this deprecation.

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.  

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.
  • Immediately dropping support of empty group id: This was ruled out for the benefit of users who unknowingly rely on the empty group id for storing offsets. They will be able to continue do that until the next major release by explicitly providing "" as their consumers' group id. They will have until the next major release to figure out an alternative option for these consumers.
  • No labels