diff --git a/metrics/src/main/java/io/confluent/common/metrics/CompoundStat.java b/metrics/src/main/java/io/confluent/common/metrics/CompoundStat.java index a23d16e08e8..078450efe42 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/CompoundStat.java +++ b/metrics/src/main/java/io/confluent/common/metrics/CompoundStat.java @@ -28,25 +28,19 @@ public interface CompoundStat extends Stat { public static class NamedMeasurable { - private final String name; - private final String description; + private final MetricName name; private final Measurable stat; - public NamedMeasurable(String name, String description, Measurable stat) { + public NamedMeasurable(MetricName name, Measurable stat) { super(); this.name = name; - this.description = description; this.stat = stat; } - public String name() { + public MetricName name() { return name; } - public String description() { - return description; - } - public Measurable stat() { return stat; } diff --git a/metrics/src/main/java/io/confluent/common/metrics/JmxReporter.java b/metrics/src/main/java/io/confluent/common/metrics/JmxReporter.java index 33345a56add..8876cfd2145 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/JmxReporter.java +++ b/metrics/src/main/java/io/confluent/common/metrics/JmxReporter.java @@ -83,19 +83,40 @@ public void metricChange(KafkaMetric metric) { private KafkaMbean addAttribute(KafkaMetric metric) { try { - String[] names = split(prefix + metric.name()); - String qualifiedName = names[0] + "." + names[1]; - if (!this.mbeans.containsKey(qualifiedName)) { - mbeans.put(qualifiedName, new KafkaMbean(names[0], names[1])); - } - KafkaMbean mbean = this.mbeans.get(qualifiedName); - mbean.setAttribute(names[2], metric); + MetricName metricName = metric.metricName(); + String mBeanName = getMBeanName(metricName); + if (!this.mbeans.containsKey(mBeanName)) + mbeans.put(mBeanName, new KafkaMbean(mBeanName)); + KafkaMbean mbean = this.mbeans.get(mBeanName); + mbean.setAttribute(metricName.name() , metric); return mbean; } catch (JMException e) { - throw new MetricsException("Error creating mbean attribute " + metric.name(), e); + throw new MetricsException( + "Error creating mbean attribute for metricName :" + metric.metricName(), e); } } + /** + * @param metricName + * @return standard JMX MBean name in the following format + * domainName:type=metricType,key1=val1,key2=val2 + */ + private String getMBeanName(MetricName metricName) { + StringBuilder mBeanName = new StringBuilder(); + mBeanName.append(prefix); + mBeanName.append(":type="); + mBeanName.append(metricName.group()); + for (Map.Entry entry : metricName.tags().entrySet()) { + if(entry.getKey().length() <= 0 || entry.getValue().length() <= 0) + continue; + mBeanName.append(","); + mBeanName.append(entry.getKey()); + mBeanName.append("="); + mBeanName.append(entry.getValue()); + } + return mBeanName.toString(); + } + public void close() { synchronized (lock) { for (KafkaMbean mbean : this.mbeans.values()) { @@ -124,32 +145,14 @@ private void reregister(KafkaMbean mbean) { } } - private String[] split(String name) { - int attributeStart = name.lastIndexOf('.'); - if (attributeStart < 0) { - throw new IllegalArgumentException("No MBean name in metric name: " + name); - } - String attributeName = name.substring(attributeStart + 1, name.length()); - String remainder = name.substring(0, attributeStart); - int beanStart = remainder.lastIndexOf('.'); - if (beanStart < 0) { - return new String[]{"", remainder, attributeName}; - } - String packageName = remainder.substring(0, beanStart); - String beanName = remainder.substring(beanStart + 1, remainder.length()); - return new String[]{packageName, beanName, attributeName}; - } - private static class KafkaMbean implements DynamicMBean { - private final String beanName; private final ObjectName objectName; private final Map metrics; - public KafkaMbean(String packageName, String beanName) throws MalformedObjectNameException { - this.beanName = beanName; + public KafkaMbean(String mbeanName) throws MalformedObjectNameException { this.metrics = new HashMap(); - this.objectName = new ObjectName(packageName + ":type=" + beanName); + this.objectName = new ObjectName(mbeanName); } public ObjectName name() { @@ -192,11 +195,11 @@ public MBeanInfo getMBeanInfo() { String attribute = entry.getKey(); KafkaMetric metric = entry.getValue(); attrs[i] = - new MBeanAttributeInfo(attribute, double.class.getName(), metric.description(), true, - false, false); + new MBeanAttributeInfo(attribute, double.class.getName(), metric.metricName() + .description(), true, false, false); i += 1; } - return new MBeanInfo(beanName, "", attrs, null, null, null); + return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null); } @Override diff --git a/metrics/src/main/java/io/confluent/common/metrics/KafkaMetric.java b/metrics/src/main/java/io/confluent/common/metrics/KafkaMetric.java index 53a3f1b9dc7..a2d89c483a9 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/KafkaMetric.java +++ b/metrics/src/main/java/io/confluent/common/metrics/KafkaMetric.java @@ -20,18 +20,16 @@ public final class KafkaMetric implements Metric { - private final String name; - private final String description; + private MetricName metricName; private final Object lock; private final Time time; private final Measurable measurable; private MetricConfig config; - KafkaMetric(Object lock, String name, String description, Measurable measurable, - MetricConfig config, Time time) { + KafkaMetric(Object lock, MetricName metricName, Measurable measurable, MetricConfig config, + Time time) { super(); - this.name = name; - this.description = description; + this.metricName = metricName; this.lock = lock; this.measurable = measurable; this.config = config; @@ -43,13 +41,8 @@ MetricConfig config() { } @Override - public String name() { - return this.name; - } - - @Override - public String description() { - return this.description; + public MetricName metricName() { + return this.metricName; } @Override diff --git a/metrics/src/main/java/io/confluent/common/metrics/Metric.java b/metrics/src/main/java/io/confluent/common/metrics/Metric.java index c298cc80624..cfe40176e3a 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/Metric.java +++ b/metrics/src/main/java/io/confluent/common/metrics/Metric.java @@ -22,14 +22,9 @@ public interface Metric { /** - * A unique name for this metric + * A name for this metric */ - public String name(); - - /** - * A description of what is measured...this will be "" if no description was given - */ - public String description(); + public MetricName metricName(); /** * The value of the metric diff --git a/metrics/src/main/java/io/confluent/common/metrics/MetricName.java b/metrics/src/main/java/io/confluent/common/metrics/MetricName.java new file mode 100644 index 00000000000..15d36eea850 --- /dev/null +++ b/metrics/src/main/java/io/confluent/common/metrics/MetricName.java @@ -0,0 +1,191 @@ +/** + * 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 io.confluent.common.metrics; + +import java.util.HashMap; +import java.util.Map; + +import io.confluent.common.utils.Utils; + +/** + * The MetricName class encapsulates a metric's name, logical group and its related attributes + *

+ * This class captures the following parameters + *

+ *  name The name of the metric
+ *  group logical group name of the metrics to which this metric belongs.
+ *  description A human-readable description to include in the metric. This is optional.
+ *  tags additional key/value attributes of the metric. This is optional.
+ * 
+ * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting. + *

+ * Ex: standard JMX MBean can be constructed like domainName:type=group,key1=val1,key2=val2 + *

+ * Usage looks something like this: + *

{@code
+ * // set up metrics:
+ * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
+ * Sensor sensor = metrics.sensor("message-sizes");
+ * Map metricTags = new LinkedHashMap();
+ * metricTags.put("client-id", "producer-1");
+ * metricTags.put("topic", "topic");
+ * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
+ * sensor.add(metricName, new Avg());
+ * metricName = new MetricName("message-size-max", "producer-metrics",metricTags);
+ * sensor.add(metricName, new Max());
+ *
+ * // as messages are sent we record the sizes
+ * sensor.record(messageSize);
+ * }
+ */ +public final class MetricName { + + private final String name; + private final String group; + private final String description; + private Map tags; + private int hash = 0; + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + * @param tags additional key/value attributes of the metric + */ + public MetricName(String name, String group, String description, Map tags) { + this.name = Utils.notNull(name); + this.group = Utils.notNull(group); + this.description = Utils.notNull(description); + this.tags = Utils.notNull(tags); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + * @param keyValue additional key/value attributes of the metric (must come in pairs) + */ + public MetricName(String name, String group, String description, String... keyValue) { + this(name, group, description, getTags(keyValue)); + } + + private static Map getTags(String... keyValue) { + if ((keyValue.length % 2) != 0) { + throw new IllegalArgumentException("keyValue needs to be specified in paris"); + } + Map tags = new HashMap(); + + for (int i = 0; i < (keyValue.length / 2); i++) { + tags.put(keyValue[i], keyValue[i + 1]); + } + return tags; + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param tags key/value attributes of the metric + */ + public MetricName(String name, String group, Map tags) { + this(name, group, "", tags); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + */ + public MetricName(String name, String group, String description) { + this(name, group, description, new HashMap()); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + */ + public MetricName(String name, String group) { + this(name, group, "", new HashMap()); + } + + public String name() { + return this.name; + } + + public String group() { + return this.group; + } + + public Map tags() { + return this.tags; + } + + public String description() { + return this.description; + } + + @Override + public int hashCode() { + if (hash != 0) { + return hash; + } + final int prime = 31; + int result = 1; + result = prime * result + ((group == null) ? 0 : group.hashCode()); + result = prime * result + ((name == null) ? 0 : name.hashCode()); + result = prime * result + ((tags == null) ? 0 : tags.hashCode()); + this.hash = result; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + MetricName other = (MetricName) obj; + if (group == null) { + if (other.group != null) { + return false; + } + } else if (!group.equals(other.group)) { + return false; + } + if (name == null) { + if (other.name != null) { + return false; + } + } else if (!name.equals(other.name)) { + return false; + } + if (tags == null) { + if (other.tags != null) { + return false; + } + } else if (!tags.equals(other.tags)) { + return false; + } + return true; + } + + @Override + public String toString() { + return "MetricName [name=" + name + ", group=" + group + ", description=" + + description + ", tags=" + tags + "]"; + } +} \ No newline at end of file diff --git a/metrics/src/main/java/io/confluent/common/metrics/Metrics.java b/metrics/src/main/java/io/confluent/common/metrics/Metrics.java index 5bab96c3ec4..6c6194f82d8 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/Metrics.java +++ b/metrics/src/main/java/io/confluent/common/metrics/Metrics.java @@ -33,8 +33,10 @@ * // set up metrics: * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors * Sensor sensor = metrics.sensor("message-sizes"); - * sensor.add("kafka.producer.message-sizes.avg", new Avg()); - * sensor.add("kafka.producer.message-sizes.max", new Max()); + * MetricName metricName = new MetricName("message-size-avg", "producer-metrics"); + * sensor.add(metricName, new Avg()); + * metricName = new MetricName("message-size-max", "producer-metrics"); + * sensor.add(metricName, new Max()); * * // as messages are sent we record the sizes * sensor.record(messageSize); @@ -43,7 +45,7 @@ public class Metrics { private final MetricConfig config; - private final ConcurrentMap metrics; + private final ConcurrentMap metrics; private final ConcurrentMap sensors; private final List reporters; private final Time time; @@ -83,7 +85,7 @@ public Metrics(MetricConfig defaultConfig) { public Metrics(MetricConfig defaultConfig, List reporters, Time time) { this.config = defaultConfig; this.sensors = new CopyOnWriteMap(); - this.metrics = new CopyOnWriteMap(); + this.metrics = new CopyOnWriteMap(); this.reporters = Utils.notNull(reporters); this.time = time; for (MetricsReporter reporter : reporters) { @@ -146,51 +148,24 @@ public synchronized Sensor sensor(String name, MetricConfig config, Sensor... pa * Add a metric to monitor an object that implements measurable. This metric won't be associated * with any sensor. This is a way to expose existing values as metrics. * - * @param name The name of the metric + * @param metricName The name of the metric * @param measurable The measurable that will be measured by this metric */ - public void addMetric(String name, Measurable measurable) { - addMetric(name, "", measurable); + public void addMetric(MetricName metricName, Measurable measurable) { + addMetric(metricName, null, measurable); } /** * Add a metric to monitor an object that implements measurable. This metric won't be associated * with any sensor. This is a way to expose existing values as metrics. - * - * @param name The name of the metric - * @param description A human-readable description to include in the metric - * @param measurable The measurable that will be measured by this metric - */ - public void addMetric(String name, String description, Measurable measurable) { - addMetric(name, description, null, measurable); - } - - /** - * Add a metric to monitor an object that implements measurable. This metric won't be associated - * with any sensor. This is a way to expose existing values as metrics. - * - * @param name The name of the metric - * @param config The configuration to use when measuring this measurable - * @param measurable The measurable that will be measured by this metric - */ - public void addMetric(String name, MetricConfig config, Measurable measurable) { - addMetric(name, "", config, measurable); - } - - /** - * Add a metric to monitor an object that implements measurable. This metric won't be associated - * with any sensor. This is a way to expose existing values as metrics. - * - * @param name The name of the metric - * @param description A human-readable description to include in the metric + * @param metricName The name of the metric * @param config The configuration to use when measuring this measurable * @param measurable The measurable that will be measured by this metric */ - public synchronized void addMetric(String name, String description, MetricConfig config, - Measurable measurable) { + public synchronized void addMetric(MetricName metricName, MetricConfig config, Measurable + measurable) { KafkaMetric m = new KafkaMetric(new Object(), - Utils.notNull(name), - Utils.notNull(description), + Utils.notNull(metricName), Utils.notNull(measurable), config == null ? this.config : config, time); @@ -206,20 +181,21 @@ public synchronized void addReporter(MetricsReporter reporter) { } synchronized void registerMetric(KafkaMetric metric) { - if (this.metrics.containsKey(metric.name())) { - throw new IllegalArgumentException( - "A metric named '" + metric.name() + "' already exists, can't register another one."); + MetricName metricName = metric.metricName(); + if (this.metrics.containsKey(metricName)) { + throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, " + + "can't register another one."); } - this.metrics.put(metric.name(), metric); + this.metrics.put(metricName, metric); for (MetricsReporter reporter : reporters) { reporter.metricChange(metric); } } /** - * Get all the metrics currently maintained indexed by metric name + * Get all the metrics currently maintained indexed by metricName */ - public Map metrics() { + public Map metrics() { return this.metrics; } diff --git a/metrics/src/main/java/io/confluent/common/metrics/Sensor.java b/metrics/src/main/java/io/confluent/common/metrics/Sensor.java index 87532098f88..d8477d7f43d 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/Sensor.java +++ b/metrics/src/main/java/io/confluent/common/metrics/Sensor.java @@ -120,7 +120,7 @@ private void checkQuotas(long timeMs) { if (quota != null) { if (!quota.acceptable(metric.value(timeMs))) { throw new QuotaViolationException( - "Metric " + metric.name() + " is in violation of its quota of " + quota.bound()); + metric.metricName() + " is in violation of its quota of " + quota.bound()); } } } @@ -145,53 +145,33 @@ public void add(CompoundStat stat) { public synchronized void add(CompoundStat stat, MetricConfig config) { this.stats.add(Utils.notNull(stat)); for (CompoundStat.NamedMeasurable m : stat.stats()) { - KafkaMetric - metric = - new KafkaMetric(this, m.name(), m.description(), m.stat(), - config == null ? this.config : config, time); + KafkaMetric metric = new KafkaMetric(this, m.name(), m.stat(), + config == null ? this.config : config, time); this.registry.registerMetric(metric); this.metrics.add(metric); } } /** - * Add a metric with default configuration and no description. Equivalent to {@link - * Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, null)} - */ - public void add(String name, MeasurableStat stat) { - add(name, stat, null); - } - - /** - * Add a metric with default configuration. Equivalent to {@link Sensor#add(String, String, - * MeasurableStat, MetricConfig) add(name, description, stat, null)} - */ - public void add(String name, String description, MeasurableStat stat) { - add(name, description, stat, null); - } - - /** - * Add a metric to this sensor with no description. Equivalent to {@link Sensor#add(String, - * String, MeasurableStat, MetricConfig) add(name, "", stat, config)} + * Register a metric with this sensor + * @param metricName The name of the metric + * @param stat The statistic to keep */ - public void add(String name, MeasurableStat stat, MetricConfig config) { - add(name, "", stat, config); + public void add(MetricName metricName, MeasurableStat stat) { + add(metricName, stat, null); } /** * Register a metric with this sensor * - * @param name The name of the metric - * @param description A description used when reporting the value + * @param metricName The name of the metric * @param stat The statistic to keep * @param config A special configuration for this metric. If null use the sensor default * configuration. */ - public synchronized void add(String name, String description, MeasurableStat stat, - MetricConfig config) { - KafkaMetric metric = new KafkaMetric(this, - Utils.notNull(name), - Utils.notNull(description), + public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) { + KafkaMetric metric = new KafkaMetric(new Object(), + Utils.notNull(metricName), Utils.notNull(stat), config == null ? this.config : config, time); diff --git a/metrics/src/main/java/io/confluent/common/metrics/stats/Percentile.java b/metrics/src/main/java/io/confluent/common/metrics/stats/Percentile.java index 1657cb67703..4245b441faf 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/stats/Percentile.java +++ b/metrics/src/main/java/io/confluent/common/metrics/stats/Percentile.java @@ -16,31 +16,23 @@ */ package io.confluent.common.metrics.stats; +import io.confluent.common.metrics.MetricName; + public class Percentile { - private final String name; - private final String description; + private final MetricName name; private final double percentile; - public Percentile(String name, double percentile) { - this(name, "", percentile); - } - - public Percentile(String name, String description, double percentile) { + public Percentile(MetricName name, double percentile) { super(); this.name = name; - this.description = description; this.percentile = percentile; } - public String name() { + public MetricName name() { return this.name; } - public String description() { - return this.description; - } - public double percentile() { return this.percentile; } diff --git a/metrics/src/main/java/io/confluent/common/metrics/stats/Percentiles.java b/metrics/src/main/java/io/confluent/common/metrics/stats/Percentiles.java index 1a99b31b5a2..bcd5ffb21fd 100644 --- a/metrics/src/main/java/io/confluent/common/metrics/stats/Percentiles.java +++ b/metrics/src/main/java/io/confluent/common/metrics/stats/Percentiles.java @@ -55,7 +55,7 @@ public List stats() { List ms = new ArrayList(this.percentiles.length); for (Percentile percentile : this.percentiles) { final double pct = percentile.percentile(); - ms.add(new NamedMeasurable(percentile.name(), percentile.description(), new Measurable() { + ms.add(new NamedMeasurable(percentile.name(), new Measurable() { public double measure(MetricConfig config, long now) { return value(config, now, pct / 100.0); } diff --git a/metrics/src/test/java/io/confluent/common/metrics/JmxReporterTest.java b/metrics/src/test/java/io/confluent/common/metrics/JmxReporterTest.java index e87d02aac20..7ec6bdbec14 100644 --- a/metrics/src/test/java/io/confluent/common/metrics/JmxReporterTest.java +++ b/metrics/src/test/java/io/confluent/common/metrics/JmxReporterTest.java @@ -28,10 +28,10 @@ public void testJmxRegistration() throws Exception { Metrics metrics = new Metrics(); metrics.addReporter(new JmxReporter()); Sensor sensor = metrics.sensor("kafka.requests"); - sensor.add("pack.bean1.avg", new Avg()); - sensor.add("pack.bean2.total", new Total()); + sensor.add(new MetricName("pack.bean1.avg", "grp1"), new Avg()); + sensor.add(new MetricName("pack.bean2.total", "grp2"), new Total()); Sensor sensor2 = metrics.sensor("kafka.blah"); - sensor2.add("pack.bean1.some", new Total()); - sensor2.add("pack.bean2.some", new Total()); + sensor2.add(new MetricName("pack.bean1.some", "grp1"), new Total()); + sensor2.add(new MetricName("pack.bean2.some", "grp1"), new Total()); } } diff --git a/metrics/src/test/java/io/confluent/common/metrics/MetricsTest.java b/metrics/src/test/java/io/confluent/common/metrics/MetricsTest.java index 21d9772afa7..9fdaf7fe67b 100644 --- a/metrics/src/test/java/io/confluent/common/metrics/MetricsTest.java +++ b/metrics/src/test/java/io/confluent/common/metrics/MetricsTest.java @@ -15,6 +15,8 @@ import org.junit.Test; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; import io.confluent.common.metrics.stats.Avg; @@ -39,23 +41,42 @@ public class MetricsTest { metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); + @Test + public void testMetricName() { + MetricName n1 = new MetricName("name", "group", "description", "key1", "value1"); + Map tags = new HashMap(); + tags.put("key1", "value1"); + MetricName n2 = new MetricName("name", "group", "description", tags); + assertEquals("metric names created in two different ways should be equal", n1, n2); + + try { + new MetricName("name", "group", "description", "key1"); + fail("Creating MetricName with an old number of keyValue should fail"); + } catch (IllegalArgumentException e) { + // this is expected + } + } + @Test public void testSimpleStats() throws Exception { ConstantMeasurable measurable = new ConstantMeasurable(); - metrics.addMetric("direct.measurable", measurable); + + metrics.addMetric( + new MetricName("direct.measurable", "grp1", + "The fraction of time an appender waits for space allocation."), measurable); Sensor s = metrics.sensor("test.sensor"); - s.add("test.avg", new Avg()); - s.add("test.max", new Max()); - s.add("test.min", new Min()); - s.add("test.rate", new Rate(TimeUnit.SECONDS)); - s.add("test.occurences", new Rate(TimeUnit.SECONDS, new Count())); - s.add("test.count", new Count()); + s.add(new MetricName("test.avg", "grp1"), new Avg()); + s.add(new MetricName("test.max", "grp1"), new Max()); + s.add(new MetricName("test.min", "grp1"), new Min()); + s.add(new MetricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS)); + s.add(new MetricName("test.occurences", "grp1"), new Rate(TimeUnit.SECONDS, new Count())); + s.add(new MetricName("test.count", "grp1"), new Count()); s.add(new Percentiles(100, -100, 100, Percentiles.BucketSizing.CONSTANT, - new Percentile("test.median", 50.0), new Percentile("test.perc99_9", - 99.9))); + new Percentile(new MetricName("test.median", "grp1"), 50.0), + new Percentile(new MetricName("test.perc99_9", "grp1"), 99.9))); Sensor s2 = metrics.sensor("test.sensor2"); - s2.add("s2.total", new Total()); + s2.add(new MetricName("s2.total", "grp1"), new Total()); s2.record(5.0); for (int i = 0; i < 10; i++) { @@ -65,31 +86,36 @@ public void testSimpleStats() throws Exception { // pretend 2 seconds passed... time.sleep(2000); - assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get("s2.total").value(), - EPS); - assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get("test.avg").value(), EPS); - assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get("test.max").value(), EPS); - assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get("test.min").value(), EPS); - assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get("test.rate").value(), EPS); - assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get("test.occurences").value(), - EPS); - assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get("test.count").value(), EPS); + assertEquals("s2 reflects the constant value", 5.0, + metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS); + assertEquals("Avg(0...9) = 4.5", 4.5, + metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS); + assertEquals("Max(0...9) = 9", 9.0, + metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); + assertEquals("Min(0...9) = 0", 0.0, + metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); + assertEquals("Rate(0...9) = 22.5", 22.5, + metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals("Occurences(0...9) = 5", 5.0, + metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", 10.0, + metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); } @Test public void testHierarchicalSensors() { Sensor parent1 = metrics.sensor("test.parent1"); - parent1.add("test.parent1.count", new Count()); + parent1.add(new MetricName("test.parent1.count", "grp1"), new Count()); Sensor parent2 = metrics.sensor("test.parent2"); - parent2.add("test.parent2.count", new Count()); + parent2.add(new MetricName("test.parent2.count", "grp1"), new Count()); Sensor child1 = metrics.sensor("test.child1", parent1, parent2); - child1.add("test.child1.count", new Count()); + child1.add(new MetricName("test.child1.count", "grp1"), new Count()); Sensor child2 = metrics.sensor("test.child2", parent1); - child2.add("test.child2.count", new Count()); + child2.add(new MetricName("test.child2.count", "grp1"), new Count()); Sensor grandchild = metrics.sensor("test.grandchild", child1); - grandchild.add("test.grandchild.count", new Count()); + grandchild.add(new MetricName("test.grandchild.count", "grp1"), new Count()); - /* increment each sensor one time */ + /* increment each sensor one time */ parent1.record(); parent2.record(); child1.record(); @@ -157,15 +183,17 @@ public void testOldDataHasNoEffect() { @Test(expected = IllegalArgumentException.class) public void testDuplicateMetricName() { - metrics.sensor("test").add("test", new Avg()); - metrics.sensor("test2").add("test", new Total()); + metrics.sensor("test").add(new MetricName("test", "grp1"), new Avg()); + metrics.sensor("test2").add(new MetricName("test", "grp1"), new Total()); } @Test public void testQuotas() { Sensor sensor = metrics.sensor("test"); - sensor.add("test1.total", new Total(), new MetricConfig().quota(Quota.lessThan(5.0))); - sensor.add("test2.total", new Total(), new MetricConfig().quota(Quota.moreThan(0.0))); + sensor.add(new MetricName("test1.total", "grp1"), new Total(), + new MetricConfig().quota(Quota.lessThan(5.0))); + sensor.add(new MetricName("test2.total", "grp1"), new Total(), + new MetricConfig().quota(Quota.moreThan(0.0))); sensor.record(5.0); try { sensor.record(1.0); @@ -173,7 +201,7 @@ public void testQuotas() { } catch (QuotaViolationException e) { // this is good } - assertEquals(6.0, metrics.metrics().get("test1.total").value(), EPS); + assertEquals(6.0, metrics.metrics().get(new MetricName("test1.total", "grp1")).value(), EPS); sensor.record(-6.0); try { sensor.record(-1.0); @@ -190,15 +218,15 @@ public void testPercentiles() { 0.0, 100.0, Percentiles.BucketSizing.CONSTANT, - new Percentile("test.p25", 25), - new Percentile("test.p50", 50), - new Percentile("test.p75", 75)); + new Percentile(new MetricName("test.p25", "grp1"), 25), + new Percentile(new MetricName("test.p50", "grp1"), 50), + new Percentile(new MetricName("test.p75", "grp1"), 75)); MetricConfig config = new MetricConfig().eventWindow(50).samples(2); Sensor sensor = metrics.sensor("test", config); sensor.add(percs); - Metric p25 = this.metrics.metrics().get("test.p25"); - Metric p50 = this.metrics.metrics().get("test.p50"); - Metric p75 = this.metrics.metrics().get("test.p75"); + Metric p25 = this.metrics.metrics().get(new MetricName("test.p25", "grp1")); + Metric p50 = this.metrics.metrics().get(new MetricName("test.p50", "grp1")); + Metric p75 = this.metrics.metrics().get(new MetricName("test.p75", "grp1")); // record two windows worth of sequential values for (int i = 0; i < buckets; i++) {