Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: Accepted

Under Discussion thread: here

Discussion Vote thread: here

JIRA:

Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-6774

...

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 immediatelyon the client, and dropping its support by the next major release. It also suggests changing the default group id to help with this phase-out plan null so that unnecessary and unintentional offset fetches and commits are avoided.

Public Interfaces

  • The default group id will also change to from "" to null so that consumers that want to use an empty ("") group id for coordinator-free consumption would have to explicitly specify that group idit.
  • A new version of OffsetCommit API will be created that does not support the The use of 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:

...

  • 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 (
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.

Code Block
titleOffsetCommit 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.

Code Block
titleOffsetCommit 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

Compatibility, Deprecation, and Migration Plan

...


  • 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.
  • Immediately Deprecating and dropping support for the use of empty group id on the broker: This was also ruled out for the benefit of users who unknowingly to support backward compatibility and allow consumers that rely on the empty existing broker behavior with respect to this group id for storing offsets. They will to 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 consumersso in the future.