storm-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From agre...@apache.org
Subject [storm] branch master updated: STORM-3627 allow using short metric names for V2 metrics tick
Date Wed, 13 May 2020 16:53:37 GMT
This is an automated email from the ASF dual-hosted git repository.

agresch pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/storm.git


The following commit(s) were added to refs/heads/master by this push:
     new a3378c0  STORM-3627 allow using short metric names for V2 metrics tick
     new 69e69d4  Merge pull request #3263 from agresch/agresch_storm_3627
a3378c0 is described below

commit a3378c09fb0bf35ee20ddc1dd5a33ace8b576eda
Author: Aaron Gresch <agresch@yahoo-inc.com>
AuthorDate: Mon May 4 14:33:28 2020 -0500

    STORM-3627 allow using short metric names for V2 metrics tick
---
 .../apache/storm/metrics2/StormMetricRegistry.java | 129 +++++++++++++--------
 1 file changed, 81 insertions(+), 48 deletions(-)

diff --git a/storm-client/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-client/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 6501a4f..3401706 100644
--- a/storm-client/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-client/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -39,8 +39,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class StormMetricRegistry {
-
     private static final Logger LOG = LoggerFactory.getLogger(StormMetricRegistry.class);
+    private static final String WORKER_METRIC_PREFIX = "storm.worker.";
+    private static final String TOPOLOGY_METRIC_PREFIX = "storm.topology.";
     
     private final MetricRegistry registry = new MetricRegistry();
     private final List<StormReporter> reporters = new ArrayList<>();
@@ -53,16 +54,16 @@ public class StormMetricRegistry {
 
     public <T> SimpleGauge<T> gauge(
         T initialValue, String name, String topologyId, String componentId, Integer taskId,
Integer port) {
-        String metricName = metricName(name, topologyId, componentId, taskId, port);
-        Gauge gauge = registry.gauge(metricName, () -> new SimpleGauge<>(initialValue));
-        saveMetricTaskIdMapping(taskId, metricName, gauge, taskIdGauges);
+        MetricNames metricNames = workerMetricName(name, topologyId, componentId, taskId,
port);
+        Gauge gauge = registry.gauge(metricNames.getLongName(), () -> new SimpleGauge<>(initialValue));
+        saveMetricTaskIdMapping(taskId, metricNames, gauge, taskIdGauges);
         return (SimpleGauge<T>) gauge;
     }
 
     public <T> Gauge<T> gauge(String name, Gauge<T> gauge, TopologyContext
context) {
-        String metricName = metricName(name, context);
-        gauge = registry.register(metricName, gauge);
-        saveMetricTaskIdMapping(context.getThisTaskId(), metricName, gauge, taskIdGauges);
+        MetricNames metricNames = topologyMetricName(name, context);
+        gauge = registry.register(metricNames.getLongName(), gauge);
+        saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, gauge, taskIdGauges);
         return gauge;
     }
 
@@ -73,82 +74,81 @@ public class StormMetricRegistry {
     }
 
     public Meter meter(String name, WorkerTopologyContext context, String componentId, Integer
taskId, String streamId) {
-        String metricName = metricName(name, context.getStormId(), componentId, streamId,
taskId, context.getThisWorkerPort());
-        Meter meter = registry.meter(metricName);
-        saveMetricTaskIdMapping(taskId, metricName, meter, taskIdMeters);
+        MetricNames metricNames = workerMetricName(name, context.getStormId(), componentId,
streamId, taskId, context.getThisWorkerPort());
+        Meter meter = registry.meter(metricNames.getLongName());
+        saveMetricTaskIdMapping(taskId, metricNames, meter, taskIdMeters);
         return meter;
     }
 
     public Meter meter(String name, TopologyContext context) {
-        String metricName = metricName(name, context);
-        Meter meter = registry.meter(metricName);
-        saveMetricTaskIdMapping(context.getThisTaskId(), metricName, meter, taskIdMeters);
+        MetricNames metricNames = topologyMetricName(name, context);
+        Meter meter = registry.meter(metricNames.getLongName());
+        saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, meter, taskIdMeters);
         return meter;
     }
 
     public Counter counter(String name, WorkerTopologyContext context, String componentId,
Integer taskId, String streamId) {
-        String metricName = metricName(name, context.getStormId(), componentId, streamId,
taskId, context.getThisWorkerPort());
-        Counter counter = registry.counter(metricName);
-        saveMetricTaskIdMapping(taskId, metricName, counter, taskIdCounters);
+        MetricNames metricNames = workerMetricName(name, context.getStormId(), componentId,
streamId, taskId, context.getThisWorkerPort());
+        Counter counter = registry.counter(metricNames.getLongName());
+        saveMetricTaskIdMapping(taskId, metricNames, counter, taskIdCounters);
         return counter;
     }
 
     public Counter counter(String name, String topologyId, String componentId, Integer taskId,
Integer workerPort, String streamId) {
-        String metricName = metricName(name, topologyId, componentId, streamId, taskId, workerPort);
-        Counter counter = registry.counter(metricName);
-        saveMetricTaskIdMapping(taskId, metricName, counter, taskIdCounters);
+        MetricNames metricNames = workerMetricName(name, topologyId, componentId, streamId,
taskId, workerPort);
+        Counter counter = registry.counter(metricNames.getLongName());
+        saveMetricTaskIdMapping(taskId, metricNames, counter, taskIdCounters);
         return counter;
     }
 
     public Counter counter(String name, TopologyContext context) {
-        String metricName = metricName(name, context);
-        Counter counter = registry.counter(metricName);
-        saveMetricTaskIdMapping(context.getThisTaskId(), metricName, counter, taskIdCounters);
+        MetricNames metricNames = topologyMetricName(name, context);
+        Counter counter = registry.counter(metricNames.getLongName());
+        saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, counter, taskIdCounters);
         return counter;
     }
 
     public void metricSet(String prefix, MetricSet set, TopologyContext context) {
-        String baseName = metricName(prefix, context);
         // Instead of registering the metrics as a set, register them individually.
         // This allows fetching the individual metrics by type (getTaskGauges())
         // to work as expected.
         for (Map.Entry<String, Metric> entry : set.getMetrics().entrySet()) {
-            String metricName = baseName + "." + entry.getKey();
-            Metric metric = registry.register(metricName, entry.getValue());
+            MetricNames metricNames = topologyMetricName(prefix + "." + entry.getKey(), context);
+            Metric metric = registry.register(metricNames.getLongName(), entry.getValue());
             if (metric instanceof Gauge) {
-                saveMetricTaskIdMapping(context.getThisTaskId(), metricName, (Gauge) metric,
taskIdGauges);
+                saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, (Gauge) metric,
taskIdGauges);
             } else if (metric instanceof Meter) {
-                saveMetricTaskIdMapping(context.getThisTaskId(), metricName, (Meter) metric,
taskIdMeters);
+                saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, (Meter) metric,
taskIdMeters);
             } else if (metric instanceof Counter) {
-                saveMetricTaskIdMapping(context.getThisTaskId(), metricName, (Counter) metric,
taskIdCounters);
+                saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, (Counter) metric,
taskIdCounters);
             } else if (metric instanceof Timer) {
-                saveMetricTaskIdMapping(context.getThisTaskId(), metricName, (Timer) metric,
taskIdTimers);
+                saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, (Timer) metric,
taskIdTimers);
             } else if (metric instanceof Histogram) {
-                saveMetricTaskIdMapping(context.getThisTaskId(), metricName, (Histogram)
metric, taskIdHistograms);
+                saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, (Histogram)
metric, taskIdHistograms);
             } else {
-                LOG.error("Unable to save taskId mapping for metric {} named {}", metric,
metricName);
+                LOG.error("Unable to save taskId mapping for metric {} named {}", metric,
metricNames.getLongName());
             }
         }
     }
 
     public Timer timer(String name, TopologyContext context) {
-        String metricName = metricName(name, context);
-        Timer timer = registry.timer(metricName);
-        saveMetricTaskIdMapping(context.getThisTaskId(), metricName, timer, taskIdTimers);
+        MetricNames metricNames = topologyMetricName(name, context);
+        Timer timer = registry.timer(metricNames.getLongName());
+        saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, timer, taskIdTimers);
         return timer;
     }
 
     public Histogram histogram(String name, TopologyContext context) {
-        String metricName = metricName(name, context);
-        Histogram histogram = registry.histogram(metricName);
-        saveMetricTaskIdMapping(context.getThisTaskId(), metricName, histogram, taskIdHistograms);
+        MetricNames metricNames = topologyMetricName(name, context);
+        Histogram histogram = registry.histogram(metricNames.getLongName());
+        saveMetricTaskIdMapping(context.getThisTaskId(), metricNames, histogram, taskIdHistograms);
         return histogram;
     }
 
-    private static <T extends Metric> void saveMetricTaskIdMapping(Integer taskId,
String name, T metric, Map<Integer,
+    private static <T extends Metric> void saveMetricTaskIdMapping(Integer taskId,
MetricNames names, T metric, Map<Integer,
             Map<String, T>> taskIdMetrics) {
         Map<String, T> metrics = taskIdMetrics.computeIfAbsent(taskId, (tid) ->
new HashMap<>());
-        metrics.put(name, metric);
+        metrics.put(names.getV2TickName(), metric);
     }
 
     private <T extends Metric> Map<String, T> getMetricNameMap(int taskId, Map<Integer,
Map<String, T>> taskIdMetrics) {
@@ -219,8 +219,9 @@ public class StormMetricRegistry {
         }
     }
 
-    private String metricName(String name, String stormId, String componentId, String streamId,
Integer taskId, Integer workerPort) {
-        StringBuilder sb = new StringBuilder("storm.worker.");
+    private MetricNames workerMetricName(String name, String stormId, String componentId,
String streamId,
+                                         Integer taskId, Integer workerPort) {
+        StringBuilder sb = new StringBuilder(WORKER_METRIC_PREFIX);
         sb.append(stormId);
         sb.append(".");
         sb.append(hostName);
@@ -234,11 +235,13 @@ public class StormMetricRegistry {
         sb.append(workerPort);
         sb.append("-");
         sb.append(name);
-        return sb.toString();
+        String longName = sb.toString();
+        MetricNames names = new MetricNames(longName, name);
+        return names;
     }
 
-    private String metricName(String name, String stormId, String componentId, Integer taskId,
Integer workerPort) {
-        StringBuilder sb = new StringBuilder("storm.worker.");
+    private MetricNames workerMetricName(String name, String stormId, String componentId,
Integer taskId, Integer workerPort) {
+        StringBuilder sb = new StringBuilder(WORKER_METRIC_PREFIX);
         sb.append(stormId);
         sb.append(".");
         sb.append(hostName);
@@ -250,11 +253,13 @@ public class StormMetricRegistry {
         sb.append(workerPort);
         sb.append("-");
         sb.append(name);
-        return sb.toString();
+        String longName = sb.toString();
+        MetricNames names = new MetricNames(longName, name);
+        return names;
     }
 
-    public String metricName(String name, TopologyContext context) {
-        StringBuilder sb = new StringBuilder("storm.topology.");
+    private MetricNames topologyMetricName(String name, TopologyContext context) {
+        StringBuilder sb = new StringBuilder(TOPOLOGY_METRIC_PREFIX);
         sb.append(context.getStormId());
         sb.append(".");
         sb.append(hostName);
@@ -266,10 +271,38 @@ public class StormMetricRegistry {
         sb.append(context.getThisWorkerPort());
         sb.append("-");
         sb.append(name);
-        return sb.toString();
+        String longName = sb.toString();
+        MetricNames names = new MetricNames(longName, name);
+        return names;
     }
 
     private String dotToUnderScore(String str) {
         return str.replace('.', '_');
     }
+
+    private static class MetricNames {
+        private String longName;
+        private String shortName;
+
+        MetricNames(String longName, String shortName) {
+            this.longName = longName;
+            this.shortName = shortName;
+        }
+
+        /**
+         * Returns the full metric name to be used for registering with the metrics registry.
+         * @return The full metric name.
+         */
+        String getLongName() {
+            return longName;
+        }
+
+        /**
+         * Returns the short metric name to be used for reporting during the V2 metrics tick.
+         * @return The V2 metrics tick name.
+         */
+        String getV2TickName() {
+            return shortName;
+        }
+    }
 }


Mime
View raw message