junrao commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1261763698
##########
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:
@philipnee: `KafkaConsumer.assign()` has the following code.
```
// make sure the offsets of topic partitions the consumer is
unsubscribing from
// are committed since there will be no following rebalance
if (coordinator != null)
this.coordinator.maybeAutoCommitOffsetsAsync(time.milliseconds());
```
1. In the old code, if the groupId is not specified, coordinator will be
null and the offset commit will be skipped. In this PR, we always call offset
commit.
2. In the old code, the implementation of `maybeAutoCommitOffsetsAsync()` is
guarded by the `ENABLE_AUTO_COMMIT_CONFIG` config. In this PR, there is no such
guard.
Are both changes expected?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]