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

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

junrao commented on code in PR #13797:
URL: https://github.com/apache/kafka/pull/13797#discussion_r1221908399


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -106,4 +114,17 @@ private boolean process(final OffsetFetchApplicationEvent event) {
         manager.addOffsetFetchRequest(event.partitions);
         return true;
     }
+
+    private boolean process(final MetadataUpdateApplicationEvent event) {
+        metadata.requestUpdateForNewTopics();

Review Comment:
   MetadataUpdateApplicationEvent is for for new topics. Could we name it clearer? Will there be a separate event for full metadata update?



##########
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

Review Comment:
   Is the TODO still needed?



##########
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());

Review Comment:
   The existing consumer seems to only update the metadata for new topics. `updateMetadata` doesn't make it clear that it's for new topics. Could we make it clear?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/MetadataUpdateApplicationEvent.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.clients.consumer.internals.events;
+
+public class MetadataUpdateApplicationEvent extends ApplicationEvent {
+
+    private final long timestamp;

Review Comment:
   `timestamp` seems unused?



##########
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:
   Is this still needed?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/DefaultBackgroundThreadTest.java:
##########
@@ -166,7 +208,7 @@ private DefaultBackgroundThread mockBackgroundThread() {
                 applicationEventsQueue,
                 backgroundEventsQueue,
                 this.errorEventHandler,
-                processor,
+            applicationEventProcessor,

Review Comment:
   indentation



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -106,4 +114,17 @@ private boolean process(final OffsetFetchApplicationEvent event) {
         manager.addOffsetFetchRequest(event.partitions);
         return true;
     }
+
+    private boolean process(final MetadataUpdateApplicationEvent event) {
+        metadata.requestUpdateForNewTopics();
+        return true;
+    }
+
+    private boolean process(final UnsubscribeApplicationEvent event) {
+        /*
+                this.coordinator.onLeavePrepare();
+                this.coordinator.maybeLeaveGroup("the consumer unsubscribed from all topics");
+         */

Review Comment:
   Should we uncomment the code?



##########
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:
   `assign()` is called for both manually and automatically assigned partitions. It seems that we only want to auto commit the offsets for the latter?



-- 
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