Table of Contents |
---|
Status
Current state: Accepted
Under Discussion thread: here
Discussion Vote thread: here
JIRA: Jira server ASF JIRA columns key,summary,type,created,updated,due,assignee,reporter,priority,status,resolution serverId 5aa69414-a9e9-3523-82ec-879b028fb15b key KAFKA-6774
...
This has been summarized in table below.
When using subscribe(...) (group management) | When using assign(...) (standalone consumer) | |
---|---|---|
group.id ="" | InvalidGroupException |
|
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 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 also change to
null
so from""
tonull
so that consumers that want to use an empty (""
) group id for coordinator-free consumption would have to explicitly specify that group idit.
Proposed Changes
This is a detailed list of improvements proposed by this KIP:
- Changing the default group id to
null
- A new
OffsetCommit
API version in which, committing offsets for group id""
leads to an error.
- 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 | |
---|---|---|
| InvalidGroupException |
|
OffsetCommit
protocol below) and
|
| |
| InvalidGroupException |
|
auto.offset.reset
config)OffsetCommit
protocol below) and enable.auto.commit
is automatically set to false
.OffsetCommit
continues to work as before for older versionsOffsetCommit
API Enhancement
The current OffsetCommit
protocol looks like this.
Code Block | ||
---|---|---|
| ||
OffsetCommit Request (Version: 3) => 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: 3) => 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_COMMIT
.
Code Block | ||
---|---|---|
| ||
OffsetCommit Request (Version: 4) => 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: 4) => throttle_time_ms [responses]
throttle_time_ms => INT32
responses => topic [partition_responses]
topic => STRING
partition_responses => partition error_code
partition => INT32
error_code => INT16 |
Code Block | ||||
---|---|---|---|---|
| ||||
...
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.
|
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. 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 default 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.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.