philipnee commented on code in PR #13797: URL: https://github.com/apache/kafka/pull/13797#discussion_r1261798307
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java: ########## @@ -522,7 +525,35 @@ public void subscribe(Collection<String> topics, ConsumerRebalanceListener callb @Override public void assign(Collection<TopicPartition> partitions) { - throw new KafkaException("method not implemented"); + if (partitions == null) { + throw new IllegalArgumentException("Topic partitions collection to assign to cannot be null"); + } + + if (partitions.isEmpty()) { + this.unsubscribe(); + return; + } + + for (TopicPartition tp : partitions) { + String topic = (tp != null) ? tp.topic() : null; + if (Utils.isBlank(topic)) + throw new IllegalArgumentException("Topic partitions to assign to cannot have null or empty topic"); + } + // TODO: implement fetcher + // fetcher.clearBufferedDataForUnassignedPartitions(partitions); + + // make sure the offsets of topic partitions the consumer is unsubscribing from + // are committed since there will be no following rebalance + commit(subscriptions.allConsumed()); Review Comment: Hey @junrao - Sorry I misunderstood your concern. To your question: If the coordinator is not available, the commit request manager won't be built and this commit() will be skipped. In the `RequestManagers.java` we've got ``` if (groupRebalanceConfig != null && groupRebalanceConfig.groupId != null) { final GroupState groupState = new GroupState(groupRebalanceConfig); coordinator = new CoordinatorRequestManager(time, logContext, retryBackoffMs, errorEventHandler, groupState.groupId); commit = new CommitRequestManager(time, logContext, subscriptions, config, coordinator, groupState); } ``` to your second question: autoCommit is still guarded by the config per this line in `CommitRequestManager.java` ``` private void maybeAutoCommit() { if (!autoCommitState.isPresent()) { return; } // autocommit otherwise } ``` On the Consumer API level, we treat sync/async commit to be the same, except sync commit waits for the completion of the future. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org