[ 
https://issues.apache.org/jira/browse/KAFKA-6123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16476361#comment-16476361
 ] 

ASF GitHub Bot commented on KAFKA-6123:
---------------------------------------

KevinLiLu closed pull request #4637: KAFKA-6123: Give MetricsReporter 
auto-generated client.id
URL: https://github.com/apache/kafka/pull/4637
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java 
b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index 3cd034eff76..ed2fa84407d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -672,15 +672,16 @@ private KafkaConsumer(ConsumerConfig config,
                     
.timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), 
TimeUnit.MILLISECONDS)
                     
.recordLevel(Sensor.RecordingLevel.forName(config.getString(ConsumerConfig.METRICS_RECORDING_LEVEL_CONFIG)))
                     .tags(metricsTags);
+            // Make sure metric reporters and interceptors get clientId
+            Map<String, Object> userProvidedConfigs = config.originals();
+            userProvidedConfigs.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
             List<MetricsReporter> reporters = 
config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
-                    MetricsReporter.class);
+                    MetricsReporter.class, userProvidedConfigs);
             reporters.add(new JmxReporter(JMX_PREFIX));
             this.metrics = new Metrics(metricConfig, reporters, time);
             this.retryBackoffMs = 
config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
 
-            // load interceptors and make sure they get clientId
-            Map<String, Object> userProvidedConfigs = config.originals();
-            userProvidedConfigs.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
+            // load interceptors
             List<ConsumerInterceptor<K, V>> interceptorList = (List) (new 
ConsumerConfig(userProvidedConfigs, 
false)).getConfiguredInstances(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG,
                     ConsumerInterceptor.class);
             this.interceptors = new ConsumerInterceptors<>(interceptorList);
diff --git 
a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index 5fc9a1b9b38..53e5565454d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -350,8 +350,10 @@ public KafkaProducer(Properties properties, Serializer<K> 
keySerializer, Seriali
                     
.timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), 
TimeUnit.MILLISECONDS)
                     
.recordLevel(Sensor.RecordingLevel.forName(config.getString(ProducerConfig.METRICS_RECORDING_LEVEL_CONFIG)))
                     .tags(metricTags);
+            // Make sure metric reporters and interceptors get clientId
+            userProvidedConfigs.put(ProducerConfig.CLIENT_ID_CONFIG, clientId);
             List<MetricsReporter> reporters = 
config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG,
-                    MetricsReporter.class);
+                    MetricsReporter.class, userProvidedConfigs);
             reporters.add(new JmxReporter(JMX_PREFIX));
             this.metrics = new Metrics(metricConfig, reporters, time);
             ProducerMetrics metricsRegistry = new 
ProducerMetrics(this.metrics);
@@ -374,8 +376,7 @@ public KafkaProducer(Properties properties, Serializer<K> 
keySerializer, Seriali
                 this.valueSerializer = ensureExtended(valueSerializer);
             }
 
-            // load interceptors and make sure they get clientId
-            userProvidedConfigs.put(ProducerConfig.CLIENT_ID_CONFIG, clientId);
+            // load interceptors
             List<ProducerInterceptor<K, V>> interceptorList = (List) (new 
ProducerConfig(userProvidedConfigs, 
false)).getConfiguredInstances(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG,
                     ProducerInterceptor.class);
             this.interceptors = new ProducerInterceptors<>(interceptorList);
diff --git 
a/clients/src/test/java/org/apache/kafka/common/metrics/AutoGeneratedClientIdTestMetricsReporter.java
 
b/clients/src/test/java/org/apache/kafka/common/metrics/AutoGeneratedClientIdTestMetricsReporter.java
new file mode 100644
index 00000000000..5bf22aa97eb
--- /dev/null
+++ 
b/clients/src/test/java/org/apache/kafka/common/metrics/AutoGeneratedClientIdTestMetricsReporter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.common.metrics;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.junit.Assert;
+
+import java.util.Map;
+
+/**
+ * Test class to verify a MetricsReporter configured for a client will receive 
the auto-generated client id.
+ */
+public class AutoGeneratedClientIdTestMetricsReporter extends 
FakeMetricsReporter {
+
+    @Override
+    public void configure(Map<String, ?> configs) {
+        String clientId = 
configs.get(CommonClientConfigs.CLIENT_ID_CONFIG).toString();
+        Assert.assertTrue(clientId.contains("consumer") || 
clientId.contains("producer"));
+    }
+}
diff --git 
a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsReporterTest.java
 
b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsReporterTest.java
new file mode 100644
index 00000000000..552acc9c1c4
--- /dev/null
+++ 
b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsReporterTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.common.metrics;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.junit.Test;
+
+import java.util.Properties;
+
+public class MetricsReporterTest {
+    @Test
+    public void testAutoGeneratedClientId() {
+        // producer auto generated client id
+        Properties producerProps = new Properties();
+        producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        
producerProps.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, 
AutoGeneratedClientIdTestMetricsReporter.class.getName());
+
+        KafkaProducer<String, String> producer = new KafkaProducer<>(
+                producerProps, new StringSerializer(), new StringSerializer());
+
+        producer.close();
+
+        // consumer auto generated client id
+        Properties consumerProps = new Properties();
+        consumerProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        
consumerProps.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, 
AutoGeneratedClientIdTestMetricsReporter.class.getName());
+
+        KafkaConsumer<String, String> consumer = new KafkaConsumer<>(
+                consumerProps, new StringDeserializer(), new 
StringDeserializer());
+
+        consumer.close();
+    }
+
+    @Test
+    public void testUserSpecifiedClientId() {
+        // producer user specified client id
+        Properties producerProps = new Properties();
+        producerProps.setProperty(ProducerConfig.CLIENT_ID_CONFIG, 
"user-specified");
+        producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        
producerProps.setProperty(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, 
UserSpecifiedClientIdTestMetricsReporter.class.getName());
+
+        KafkaProducer<String, String> producer = new KafkaProducer<>(
+                producerProps, new StringSerializer(), new StringSerializer());
+
+        producer.close();
+
+        // consumer user specified client id
+        Properties consumerProps = new Properties();
+        consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, 
"user-specified");
+        consumerProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
"localhost:9999");
+        
consumerProps.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, 
UserSpecifiedClientIdTestMetricsReporter.class.getName());
+
+        KafkaConsumer<String, String> consumer = new KafkaConsumer<>(
+                consumerProps, new StringDeserializer(), new 
StringDeserializer());
+
+        consumer.close();
+    }
+}
diff --git 
a/clients/src/test/java/org/apache/kafka/common/metrics/UserSpecifiedClientIdTestMetricsReporter.java
 
b/clients/src/test/java/org/apache/kafka/common/metrics/UserSpecifiedClientIdTestMetricsReporter.java
new file mode 100644
index 00000000000..c529e7dd1ba
--- /dev/null
+++ 
b/clients/src/test/java/org/apache/kafka/common/metrics/UserSpecifiedClientIdTestMetricsReporter.java
@@ -0,0 +1,31 @@
+/*
+ * 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.common.metrics;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.junit.Assert;
+
+import java.util.Map;
+
+public class UserSpecifiedClientIdTestMetricsReporter extends 
FakeMetricsReporter {
+
+    @Override
+    public void configure(Map<String, ?> configs) {
+        
Assert.assertFalse(configs.get(CommonClientConfigs.CLIENT_ID_CONFIG).toString().contains("producer"));
+        
Assert.assertFalse(configs.get(CommonClientConfigs.CLIENT_ID_CONFIG).toString().contains("consumer"));
+    }
+}


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Give client MetricsReporter auto-generated client.id
> ----------------------------------------------------
>
>                 Key: KAFKA-6123
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6123
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, metrics
>            Reporter: Kevin Lu
>            Priority: Minor
>              Labels: clients, metrics, newbie++
>
> KAFKA-4756 bugfix resolved the broker's KafkaMetricsReporter missing auto 
> generated broker ids, but this was not fixed on the client side.
>  
> Metric reporters configured for clients should also be given the 
> auto-generated client id in the `configure` method. The interceptors do 
> receive the auto-generated client id.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to