[ 
https://issues.apache.org/jira/browse/GOBBLIN-876?focusedWorklogId=309354&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-309354
 ]

ASF GitHub Bot logged work on GOBBLIN-876:
------------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/Sep/19 23:20
            Start Date: 09/Sep/19 23:20
    Worklog Time Spent: 10m 
      Work Description: sv2000 commented on pull request #2730: GOBBLIN-876: 
Expose metrics() API in GobblinKafkaConsumerClient to al…
URL: https://github.com/apache/incubator-gobblin/pull/2730#discussion_r322494096
 
 

 ##########
 File path: 
gobblin-modules/gobblin-kafka-09/src/main/java/org/apache/gobblin/kafka/client/Kafka09ConsumerClient.java
 ##########
 @@ -162,6 +170,35 @@ public KafkaConsumerRecord apply(ConsumerRecord<K, V> 
input) {
     });
   }
 
+  @Override
+  public Map<String, Metric> metrics() {
+    Map<MetricName, KafkaMetric> kafkaMetrics = (Map<MetricName, KafkaMetric>) 
this.consumer.metrics();
+    Map<String, Metric> codaHaleMetricMap = new HashMap<>();
+
+    kafkaMetrics
+        .forEach((key, value) -> 
codaHaleMetricMap.put(canonicalMerticName(value), 
kafkaToCodaHaleMetric(value)));
+    return codaHaleMetricMap;
+  }
+
+  /**
+   * Convert a {@link KafkaMetric} instance to a {@link Metric}.
+   * @param kafkaMetric
+   * @return
+   */
+  private Metric kafkaToCodaHaleMetric(final KafkaMetric kafkaMetric) {
+    if (log.isDebugEnabled()) {
+      log.debug("Processing a metric change for {}", 
kafkaMetric.metricName().toString());
+    }
+    Gauge<Double> gauge = () -> kafkaMetric.value();
+    return gauge;
+  }
+
+  private String canonicalMerticName(KafkaMetric kafkaMetric) {
+    MetricName name = kafkaMetric.metricName();
+    return canonicalMetricName(name.group(), name.tags().values(), 
name.name());
 
 Review comment:
   Yeah - thought about it. Even if we cache the metric name mappings, we need 
to compute the key for the map every time. It seems like the key needs to be 
derived off of metric group, metric name and tags to make the metric unique, 
which is essentially the canonical representation of the kafka metric.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 309354)
    Time Spent: 2h  (was: 1h 50m)

> Expose metrics() API in GobblinKafkaConsumerClient to allow consume metrics 
> to be reported
> ------------------------------------------------------------------------------------------
>
>                 Key: GOBBLIN-876
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-876
>             Project: Apache Gobblin
>          Issue Type: Improvement
>          Components: gobblin-kafka
>    Affects Versions: 0.15.0
>            Reporter: Sudarshan Vasudevan
>            Assignee: Shirshanka Das
>            Priority: Major
>             Fix For: 0.15.0
>
>          Time Spent: 2h
>  Remaining Estimate: 0h
>
> Newer Kafka consumer expose metrics() API that report a number of consumer 
> metrics such as lag, latency, etc. which are very useful for monitoring and 
> debugging. We expose a metrics() API in GobblinKafkaConsumerClient to allow 
> consumer metrics to be reported. 



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to