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

...

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

  • If the consumer subscribes to a topic using the default group id ("") it 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 "").
    Code Block
    languagejava
    titleInvalid use of consumer with default group id
    String topic = "foo"; Properties properties = new Properties(); properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, :
    Exception in thread "main" org.apache.kafka.common.serialization.StringDeserializer"); properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); KafkaConsumer consumer = new KafkaConsumer<String, String>(properties); consumer.subscribe(Collections.singleton(topic)); while (true) { ConsumerRecords<String, String> records = consumer.poll(100); for (ConsumerRecord record: records) System.out.println(String.format("%s: %s", record.key(), record.value())); }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 and , fetch messages, and commit offsets without having to worry about the empty string group id. Code Block
    languagejava
    titleValid use of consumer with default group id
    String topic = "foo"; Properties properties = new Properties(); properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); properties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); properties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); KafkaConsumer consumer = new KafkaConsumer<String, String>(properties); TopicPartition tp = new TopicPartition(topic, 0); consumer.assign(Arrays.asList(tp)); consumer.seekToBeginning(Arrays.asList(tp)); System.out.println(consumer.position(tp)); consumer.seek(tp, 5); System.out.println(consumer.position(tp)); while (true) { ConsumerRecords<String, String> records = consumer.poll(100); for (ConsumerRecord record: records) System.out.println(String.format("%s: %s", record.key(), record.value())); }

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 empty string group id is of "" is troublesome, and consumers are not normally expected to do that in a production setting. That is why this KIP suggests some limitation to be enforced in this situationThis 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

KafkaConsumer semantics will be slightly modified for the stand-alone consumer (that uses the assign(...) interface) to disallow committing offsets for the default group id ("").

Proposed Changes

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.

Code Block
languagejava
titleOffset commit error when standalone consumer commits offsets in a group with id=""
    ...
    ILLEGAL_OFFSET_COMMIT(72, "The standalone consumer belonging to a group with default id (\"\") is not allowed to commit offsets",
        new ApiExceptionBuilder() {
            @Override
            public ApiException build(String message) {
                return new IllegalOffsetCommitException(message);
            }
    }
    ...
  • 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. 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

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.

Rejected Alternatives

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 futureChanging the default group id from "" to null: This means still allowing the empty string be allowed for offset commit, and stand-alone consumers that relied on "" as their group id will be required to specifically specify that group id in consumer configuration. The option proposed in the KIP seems to be cleaner as it disallows a bad practice and should have minimal to no backward compatibility impact.