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

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

Reply via email to