dajac commented on code in PR #14565:
URL: https://github.com/apache/kafka/pull/14565#discussion_r1368427893


##########
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java:
##########
@@ -961,7 +960,7 @@ public void subscribe(Collection<String> topics, 
ConsumerRebalanceListener liste
      */
     @Override
     public void subscribe(Collection<String> topics) {
-        subscribe(topics, new NoOpConsumerRebalanceListener());
+        subscribe(topics, null);

Review Comment:
   The javadoc of `subscribe` says:
   
   ```
        * @param listener Non-null listener instance to get notifications on 
partition assignment/revocation for the
        *                 subscribed topics
        * @throws IllegalArgumentException If topics is null or contains null 
or empty elements, or if listener is null
   ```
   
   With these changes, we break the contract. We should try to maintain it. 
What do you think?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -227,6 +224,12 @@ public ConsumerCoordinator(GroupRebalanceConfig 
rebalanceConfig,
             protocol = null;
         }
 
+        this.rebalanceListenerInvoker = new ConsumerRebalanceListenerInvoker(
+                logContext,
+                subscriptions,
+                time,
+                coordinatorMetrics

Review Comment:
   nit: I think that we indent arguments with 4 spaces in this case.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerRebalanceListenerInvoker.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.InterruptException;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+
+/**
+ * This class encapsulates the invocation of the callback methods defined in 
the {@link ConsumerRebalanceListener}
+ * interface. When consumer group partition assignment changes, these methods 
are invoked. This class wraps those
+ * callback calls with some logging, optional {@link Sensor} updates, etc.
+ */
+class ConsumerRebalanceListenerInvoker {
+
+    private final Logger log;
+    private final SubscriptionState subscriptions;
+    private final Time time;
+    private final ConsumerCoordinatorMetrics coordinatorMetrics;
+
+    ConsumerRebalanceListenerInvoker(LogContext logContext,
+                                     SubscriptionState subscriptions,
+                                     Time time,
+                                     ConsumerCoordinatorMetrics 
coordinatorMetrics) {
+        this.log = logContext.logger(getClass());
+        this.subscriptions = subscriptions;
+        this.time = time;
+        this.coordinatorMetrics = coordinatorMetrics;
+    }
+
+    Exception invokePartitionsAssigned(final SortedSet<TopicPartition> 
assignedPartitions) {
+        log.info("Adding newly assigned partitions: {}", 
Utils.join(assignedPartitions, ", "));
+
+        Optional<ConsumerRebalanceListener> listener = 
subscriptions.rebalanceListener();
+
+        if (listener.isPresent()) {
+            try {
+                final long startMs = time.milliseconds();
+                listener.get().onPartitionsAssigned(assignedPartitions);
+                
coordinatorMetrics.assignCallbackSensor.record(time.milliseconds() - startMs);
+            } catch (WakeupException | InterruptException e) {
+                throw e;
+            } catch (Exception e) {
+                log.error("User provided listener {} failed on invocation of 
onPartitionsAssigned for partitions {}",
+                        listener.getClass().getName(), assignedPartitions, e);
+                return e;
+            }
+        }
+
+        return null;
+    }
+
+    Exception invokePartitionsRevoked(final SortedSet<TopicPartition> 
revokedPartitions) {
+        log.info("Revoke previously assigned partitions {}", 
Utils.join(revokedPartitions, ", "));
+        Set<TopicPartition> revokePausedPartitions = 
subscriptions.pausedPartitions();
+        revokePausedPartitions.retainAll(revokedPartitions);
+        if (!revokePausedPartitions.isEmpty())
+            log.info("The pause flag in partitions [{}] will be removed due to 
revocation.", org.apache.kafka.common.utils.Utils.join(revokePausedPartitions, 
", "));

Review Comment:
   nit: Could we remove `org.apache.kafka.common.utils.`?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java:
##########
@@ -143,7 +143,7 @@
 public class FetcherTest {
     private static final double EPSILON = 0.0001;
 
-    private ConsumerRebalanceListener listener = new 
NoOpConsumerRebalanceListener();
+    private Optional<ConsumerRebalanceListener> listener = Optional.empty();

Review Comment:
   nit: Should we directly remove `listener`?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorMetrics.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Meter;
+import org.apache.kafka.common.metrics.stats.WindowedCount;
+
+class ConsumerCoordinatorMetrics {
+
+    final Sensor commitSensor;
+    final Sensor revokeCallbackSensor;
+    final Sensor assignCallbackSensor;
+    final Sensor loseCallbackSensor;
+
+    ConsumerCoordinatorMetrics(SubscriptionState subscriptions,
+                               Metrics metrics,
+                               String metricGrpPrefix) {
+        String metricGrpName = metricGrpPrefix + "-coordinator-metrics";
+
+        this.commitSensor = metrics.sensor("commit-latency");
+        this.commitSensor.add(metrics.metricName("commit-latency-avg",
+                metricGrpName,
+                "The average time taken for a commit request"), new Avg());
+        this.commitSensor.add(metrics.metricName("commit-latency-max",
+                metricGrpName,
+                "The max time taken for a commit request"), new Max());
+        this.commitSensor.add(new Meter(new WindowedCount(),
+                metrics.metricName("commit-rate", metricGrpName,
+                        "The number of commit calls per second"),
+                metrics.metricName("commit-total", metricGrpName,
+                        "The total number of commit calls")));
+        this.revokeCallbackSensor = 
metrics.sensor("partition-revoked-latency");

Review Comment:
   nit: Let's add an empty line before L51.



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