You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/11/10 22:47:41 UTC

[GitHub] [kafka] brianwyka opened a new pull request #9584: Add group-id to metrics for KafkaConsumer

brianwyka opened a new pull request #9584:
URL: https://github.com/apache/kafka/pull/9584


   Add the "group-id" to the metrics created curing instantiation of the `KafkaConsumer`
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

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



[GitHub] [kafka] kirktrue commented on a change in pull request #9584: [KAFKA-10708]: Add "group-id" Tag to Kafka Consumer Metrics

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #9584:
URL: https://github.com/apache/kafka/pull/9584#discussion_r780603531



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
##########
@@ -858,14 +862,23 @@ public KafkaConsumer(Map<String, Object> configs,
         this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
     }
 
-    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId) {
-        Map<String, String> metricsTags = Collections.singletonMap(CLIENT_ID_METRIC_TAG, clientId);
+    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId, String groupId) {
+        Map<String, String> metricsTags = new HashMap<>();
+        metricsTags.put(CLIENT_ID_METRIC_TAG, clientId);
+        if (groupId != null && !groupId.trim().isEmpty()) {

Review comment:
       The secondary `!groupId.trim().isEmpty()` clause of the `if` statement shouldn't ever catch given that that case is  checked for in the constructor before calling `buildMetrics`, right?

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
##########
@@ -858,14 +862,23 @@ public KafkaConsumer(Map<String, Object> configs,
         this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
     }
 
-    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId) {
-        Map<String, String> metricsTags = Collections.singletonMap(CLIENT_ID_METRIC_TAG, clientId);
+    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId, String groupId) {
+        Map<String, String> metricsTags = new HashMap<>();
+        metricsTags.put(CLIENT_ID_METRIC_TAG, clientId);
+        if (groupId != null && !groupId.trim().isEmpty()) {
+            metricsTags.put(GROUP_ID_METRIC_TAG, groupId);
+        }
         MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_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);
+        final Map<String, Object> reporterConfigOverrides = new HashMap<>();
+        reporterConfigOverrides.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
+        if (groupId != null && !groupId.trim().isEmpty()) {

Review comment:
       Same minor nitpick about whether or not we need to check for an empty group ID.




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



[GitHub] [kafka] brianwyka removed a comment on pull request #9584: [KAFKA-10708]: Add "group-id" Tag to Kafka Consumer Metrics

Posted by GitBox <gi...@apache.org>.
brianwyka removed a comment on pull request #9584:
URL: https://github.com/apache/kafka/pull/9584#issuecomment-725016649


   I couldn't find any tests that are currently testing out the `client-id` tag for reference.  Would appreciate a point in the right direction.


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

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



[GitHub] [kafka] brianwyka commented on pull request #9584: Add group-id to metrics for KafkaConsumer

Posted by GitBox <gi...@apache.org>.
brianwyka commented on pull request #9584:
URL: https://github.com/apache/kafka/pull/9584#issuecomment-725016649


   I couldn't find any tests that are currently testing out the `client-id` tag for reference.  Would appreciate a point in the right direction.


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

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



[GitHub] [kafka] kirktrue commented on a change in pull request #9584: [KAFKA-10708]: Add "group-id" Tag to Kafka Consumer Metrics

Posted by GitBox <gi...@apache.org>.
kirktrue commented on a change in pull request #9584:
URL: https://github.com/apache/kafka/pull/9584#discussion_r780603531



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
##########
@@ -858,14 +862,23 @@ public KafkaConsumer(Map<String, Object> configs,
         this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
     }
 
-    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId) {
-        Map<String, String> metricsTags = Collections.singletonMap(CLIENT_ID_METRIC_TAG, clientId);
+    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId, String groupId) {
+        Map<String, String> metricsTags = new HashMap<>();
+        metricsTags.put(CLIENT_ID_METRIC_TAG, clientId);
+        if (groupId != null && !groupId.trim().isEmpty()) {

Review comment:
       The secondary `!groupId.trim().isEmpty()` clause of the `if` statement shouldn't ever catch given that that case is  checked for in the constructor before calling `buildMetrics`, right?

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
##########
@@ -858,14 +862,23 @@ public KafkaConsumer(Map<String, Object> configs,
         this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer");
     }
 
-    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId) {
-        Map<String, String> metricsTags = Collections.singletonMap(CLIENT_ID_METRIC_TAG, clientId);
+    private static Metrics buildMetrics(ConsumerConfig config, Time time, String clientId, String groupId) {
+        Map<String, String> metricsTags = new HashMap<>();
+        metricsTags.put(CLIENT_ID_METRIC_TAG, clientId);
+        if (groupId != null && !groupId.trim().isEmpty()) {
+            metricsTags.put(GROUP_ID_METRIC_TAG, groupId);
+        }
         MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_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);
+        final Map<String, Object> reporterConfigOverrides = new HashMap<>();
+        reporterConfigOverrides.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
+        if (groupId != null && !groupId.trim().isEmpty()) {

Review comment:
       Same minor nitpick about whether or not we need to check for an empty group ID.




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



[GitHub] [kafka] brianwyka removed a comment on pull request #9584: [KAFKA-10708]: Add "group-id" Tag to Kafka Consumer Metrics

Posted by GitBox <gi...@apache.org>.
brianwyka removed a comment on pull request #9584:
URL: https://github.com/apache/kafka/pull/9584#issuecomment-725016649


   I couldn't find any tests that are currently testing out the `client-id` tag for reference.  Would appreciate a point in the right direction.


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

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



[GitHub] [kafka] brianwyka commented on pull request #9584: [KAFKA-10708]: Add "group-id" Tag to Kafka Consumer Metrics

Posted by GitBox <gi...@apache.org>.
brianwyka commented on pull request #9584:
URL: https://github.com/apache/kafka/pull/9584#issuecomment-1009032105


   I'm not sure @kirktrue.


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