Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-9410: Make groupId Optional in KafkaConsumer #7943

Merged
merged 1 commit into from
Jan 16, 2020
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -570,7 +570,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {

private Logger log;
private final String clientId;
private String groupId;
private final Optional<String> groupId;
private final ConsumerCoordinator coordinator;
private final Deserializer<K> keyDeserializer;
private final Deserializer<V> valueDeserializer;
Expand Down Expand Up @@ -673,28 +673,30 @@ private KafkaConsumer(ConsumerConfig config, Deserializer<K> keyDeserializer, De
GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig(config,
GroupRebalanceConfig.ProtocolType.CONSUMER);

this.groupId = groupRebalanceConfig.groupId;
this.groupId = Optional.ofNullable(groupRebalanceConfig.groupId);
this.clientId = buildClientId(config.getString(CommonClientConfigs.CLIENT_ID_CONFIG), groupRebalanceConfig);

LogContext logContext;

// If group.instance.id is set, we will append it to the log context.
if (groupRebalanceConfig.groupInstanceId.isPresent()) {
logContext = new LogContext("[Consumer instanceId=" + groupRebalanceConfig.groupInstanceId.get() +
", clientId=" + clientId + ", groupId=" + groupId + "] ");
", clientId=" + clientId + ", groupId=" + groupId.orElse("null") + "] ");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps we can improve this. If the groupId is null, perhaps we can leave it out of the log context?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I admit that 'null' is not flattering, but I was preserving the current behavior. Would you like something else? We can improve this is a follow-up JIRA if you'd like to discuss.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can do it separately if you like.

} else {
logContext = new LogContext("[Consumer clientId=" + clientId + ", groupId=" + groupId + "] ");
logContext = new LogContext("[Consumer clientId=" + clientId + ", groupId=" + groupId.orElse("null") + "] ");
}

this.log = logContext.logger(getClass());
boolean enableAutoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
if (groupId == null) { // overwrite in case of default group id where the config is not explicitly provided
if (!config.originals().containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG))
if (!groupId.isPresent()) { // overwrite in case of default group id where the config is not explicitly provided
if (!config.originals().containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) {
enableAutoCommit = false;
else if (enableAutoCommit)
} else if (enableAutoCommit) {
throw new InvalidConfigurationException(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + " cannot be set to true when default group id (null) is used.");
} else if (groupId.isEmpty())
}
} else if (groupId.get().isEmpty()) {
log.warn("Support for using the empty group id by consumers is deprecated and will be removed in the next major release.");
}

log.debug("Initializing the Kafka consumer");
this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
Expand Down Expand Up @@ -773,7 +775,7 @@ else if (enableAutoCommit)
this.assignors = getAssignorInstances(config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), config.originals());

// no coordinator will be constructed for the default (null) group id
this.coordinator = groupId == null ? null :
this.coordinator = !groupId.isPresent() ? null :
new ConsumerCoordinator(groupRebalanceConfig,
logContext,
this.client,
Expand Down Expand Up @@ -858,7 +860,7 @@ else if (enableAutoCommit)
this.requestTimeoutMs = requestTimeoutMs;
this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.assignors = assignors;
this.groupId = groupId;
this.groupId = Optional.ofNullable(groupId);
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
}

Expand Down Expand Up @@ -2436,7 +2438,7 @@ private void throwIfNoAssignorsConfigured() {
}

private void maybeThrowInvalidGroupIdException() {
if (groupId == null)
if (!groupId.isPresent())
throw new InvalidGroupIdException("To use the group management or offset commit APIs, you must " +
"provide a valid " + ConsumerConfig.GROUP_ID_CONFIG + " in the consumer configuration.");
}
Expand Down