Versions Compared

Key

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

...

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. While in the long term the usage of "" as group id should be disallowed, this KIP suggests some improvements to how this group id works.

Public Interfaces

...

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  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, committing offsets for the group id "" leads to an error 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

...


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

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

group.id=null (default)

InvalidGroupException
  • Cannot fetch offsets (uses auto.offset.reset config)
  • 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
  • Can seek to and fetch from offset (like before)
  • Cannot fetch offset (uses auto.offset.reset config)
  • Can no longer commit offset (see updated OffsetCommit protocol below) and enable.auto.commit is automatically set to false.
    OffsetCommit continues to work as before for older versions

OffsetCommit API Enhancement

...

Code Block
titleOffsetCommit Protocol - Version 35
OffsetCommit Request (Version: 35) => 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: 35) => 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 a new type of error: ILLEGAL_OFFSET_COMMITin INVALID_GROUP_ID.

code
Code Block
titleOffsetCommit Protocol - Version 46
OffsetCommit Request (Version: 46) => 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: 46) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code 
      partition => INT32
      error_code => INT16
languagejava
titleOffset commit error when standalone consumer commits offsets in a group with id=""
    ...
    ILLEGAL_OFFSET_COMMIT(73, "The standalone consumer belonging to an empty (\"\") or null group id is not allowed to commit offsets",
        new ApiExceptionBuilder() {
            @Override
            public ApiException build(String message) {
                return new IllegalOffsetCommitException(message);
            }
    }
    ...

Consumers with the default group id ("") that use assign(...) to consume from hand-picked partitions will not be able to commit offsets. If they attempt an offset commit they will receive an error. A new error will be introduced in org.apache.kafka.common.protocol.Errors for this purpose.

Stand-alone consumers using the default group id will have to seek to an offset before start fetching; or the auto.offset.commit value will be used to determine the starting point of fetch. They will no longer be able to start from a previously committed fetch position.

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 server-side warning will be logged to indicate this deprecation.

Stand-alone consumers that used the default group id for fetching messages and committing offsets would no longer be able to do that. This should be acceptable because, in general, consumers are not expected to use the default group id for offset commit, which is a dangerous and troublesome practice. This KIP aims at closing the existing gap in using the default group id. Therefore, it is advised that these consumers start using valid group names for committing offsets.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.