You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "kirktrue (via GitHub)" <gi...@apache.org> on 2023/04/28 22:07:32 UTC

[GitHub] [kafka] kirktrue commented on a diff in pull request #13605: KAFKA-14950: implement assign() and assignment()

kirktrue commented on code in PR #13605:
URL: https://github.com/apache/kafka/pull/13605#discussion_r1180848020


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/PrototypeAsyncConsumer.java:
##########
@@ -537,7 +568,9 @@ public void subscribe(Pattern pattern) {
 
     @Override
     public void unsubscribe() {
-        throw new KafkaException("method not implemented");
+        // fetcher.clearBufferedDataForUnassignedPartitions(Collections.emptySet());

Review Comment:
   I'm wondering if we want to push the clearing of buffered data and `SubscriptionState` mutation into the background thread so that it's executed together?
   
   The coordinator's `onLeavePrepare` needs to know what partitions are being removed. As it's written, it's possible that the set of partitions could be removed before the background thread gets a chance to run the `onLeavePrepare` logic. 



##########
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());
+
+        log.info("Assigned to partition(s): {}", Utils.join(partitions, ", "));
+        if (this.subscriptions.assignFromUser(new HashSet<>(partitions)))
+           updateMetadata(time.milliseconds());
+    }
+
+    private void updateMetadata(long milliseconds) {

Review Comment:
   Nitpicky: can we fold `updateMetadata` into `assign`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org