001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.metrics2.lib;
020
021import java.util.Collection;
022import java.util.Map;
023
024import com.google.common.collect.Maps;
025import com.google.common.base.Objects;
026
027import org.apache.hadoop.classification.InterfaceAudience;
028import org.apache.hadoop.classification.InterfaceStability;
029import org.apache.hadoop.metrics2.MetricsInfo;
030import org.apache.hadoop.metrics2.MetricsException;
031import org.apache.hadoop.metrics2.MetricsRecordBuilder;
032import org.apache.hadoop.metrics2.MetricsTag;
033import org.apache.hadoop.metrics2.impl.MsInfo;
034
035/**
036 * An optional metrics registry class for creating and maintaining a
037 * collection of MetricsMutables, making writing metrics source easier.
038 */
039@InterfaceAudience.Public
040@InterfaceStability.Evolving
041public class MetricsRegistry {
042  private final Map<String, MutableMetric> metricsMap = Maps.newLinkedHashMap();
043  private final Map<String, MetricsTag> tagsMap = Maps.newLinkedHashMap();
044  private final MetricsInfo metricsInfo;
045
046  /**
047   * Construct the registry with a record name
048   * @param name  of the record of the metrics
049   */
050  public MetricsRegistry(String name) {
051    metricsInfo = Interns.info(name, name);
052  }
053
054  /**
055   * Construct the registry with a metadata object
056   * @param info  the info object for the metrics record/group
057   */
058  public MetricsRegistry(MetricsInfo info) {
059    metricsInfo = info;
060  }
061
062  /**
063   * @return the info object of the metrics registry
064   */
065  public MetricsInfo info() {
066    return metricsInfo;
067  }
068
069  /**
070   * Get a metric by name
071   * @param name  of the metric
072   * @return the metric object
073   */
074  public synchronized MutableMetric get(String name) {
075    return metricsMap.get(name);
076  }
077
078  /**
079   * Get a tag by name
080   * @param name  of the tag
081   * @return the tag object
082   */
083  public synchronized MetricsTag getTag(String name) {
084    return tagsMap.get(name);
085  }
086
087  /**
088   * Create a mutable integer counter
089   * @param name  of the metric
090   * @param desc  metric description
091   * @param iVal  initial value
092   * @return a new counter object
093   */
094  public MutableCounterInt newCounter(String name, String desc, int iVal) {
095    return newCounter(Interns.info(name, desc), iVal);
096  }
097
098  /**
099   * Create a mutable integer counter
100   * @param info  metadata of the metric
101   * @param iVal  initial value
102   * @return a new counter object
103   */
104  public synchronized MutableCounterInt newCounter(MetricsInfo info, int iVal) {
105    checkMetricName(info.name());
106    MutableCounterInt ret = new MutableCounterInt(info, iVal);
107    metricsMap.put(info.name(), ret);
108    return ret;
109  }
110
111  /**
112   * Create a mutable long integer counter
113   * @param name  of the metric
114   * @param desc  metric description
115   * @param iVal  initial value
116   * @return a new counter object
117   */
118  public MutableCounterLong newCounter(String name, String desc, long iVal) {
119    return newCounter(Interns.info(name, desc), iVal);
120  }
121
122  /**
123   * Create a mutable long integer counter
124   * @param info  metadata of the metric
125   * @param iVal  initial value
126   * @return a new counter object
127   */
128  public synchronized
129  MutableCounterLong newCounter(MetricsInfo info, long iVal) {
130    checkMetricName(info.name());
131    MutableCounterLong ret = new MutableCounterLong(info, iVal);
132    metricsMap.put(info.name(), ret);
133    return ret;
134  }
135
136  /**
137   * Create a mutable integer gauge
138   * @param name  of the metric
139   * @param desc  metric description
140   * @param iVal  initial value
141   * @return a new gauge object
142   */
143  public MutableGaugeInt newGauge(String name, String desc, int iVal) {
144    return newGauge(Interns.info(name, desc), iVal);
145  }
146  /**
147   * Create a mutable integer gauge
148   * @param info  metadata of the metric
149   * @param iVal  initial value
150   * @return a new gauge object
151   */
152  public synchronized MutableGaugeInt newGauge(MetricsInfo info, int iVal) {
153    checkMetricName(info.name());
154    MutableGaugeInt ret = new MutableGaugeInt(info, iVal);
155    metricsMap.put(info.name(), ret);
156    return ret;
157  }
158
159  /**
160   * Create a mutable long integer gauge
161   * @param name  of the metric
162   * @param desc  metric description
163   * @param iVal  initial value
164   * @return a new gauge object
165   */
166  public MutableGaugeLong newGauge(String name, String desc, long iVal) {
167    return newGauge(Interns.info(name, desc), iVal);
168  }
169
170  /**
171   * Create a mutable long integer gauge
172   * @param info  metadata of the metric
173   * @param iVal  initial value
174   * @return a new gauge object
175   */
176  public synchronized MutableGaugeLong newGauge(MetricsInfo info, long iVal) {
177    checkMetricName(info.name());
178    MutableGaugeLong ret = new MutableGaugeLong(info, iVal);
179    metricsMap.put(info.name(), ret);
180    return ret;
181  }
182
183  /**
184   * Create a mutable metric that estimates quantiles of a stream of values
185   * @param name of the metric
186   * @param desc metric description
187   * @param sampleName of the metric (e.g., "Ops")
188   * @param valueName of the metric (e.g., "Time" or "Latency")
189   * @param interval rollover interval of estimator in seconds
190   * @return a new quantile estimator object
191   */
192  public synchronized MutableQuantiles newQuantiles(String name, String desc,
193      String sampleName, String valueName, int interval) {
194    checkMetricName(name);
195    MutableQuantiles ret = 
196        new MutableQuantiles(name, desc, sampleName, valueName, interval);
197    metricsMap.put(name, ret);
198    return ret;
199  }
200  
201  /**
202   * Create a mutable metric with stats
203   * @param name  of the metric
204   * @param desc  metric description
205   * @param sampleName  of the metric (e.g., "Ops")
206   * @param valueName   of the metric (e.g., "Time" or "Latency")
207   * @param extended    produce extended stat (stdev, min/max etc.) if true.
208   * @return a new mutable stat metric object
209   */
210  public synchronized MutableStat newStat(String name, String desc,
211      String sampleName, String valueName, boolean extended) {
212    checkMetricName(name);
213    MutableStat ret =
214        new MutableStat(name, desc, sampleName, valueName, extended);
215    metricsMap.put(name, ret);
216    return ret;
217  }
218
219  /**
220   * Create a mutable metric with stats
221   * @param name  of the metric
222   * @param desc  metric description
223   * @param sampleName  of the metric (e.g., "Ops")
224   * @param valueName   of the metric (e.g., "Time" or "Latency")
225   * @return a new mutable metric object
226   */
227  public MutableStat newStat(String name, String desc,
228                             String sampleName, String valueName) {
229    return newStat(name, desc, sampleName, valueName, false);
230  }
231
232  /**
233   * Create a mutable rate metric
234   * @param name  of the metric
235   * @return a new mutable metric object
236   */
237  public MutableRate newRate(String name) {
238    return newRate(name, name, false);
239  }
240
241  /**
242   * Create a mutable rate metric
243   * @param name  of the metric
244   * @param description of the metric
245   * @return a new mutable rate metric object
246   */
247  public MutableRate newRate(String name, String description) {
248    return newRate(name, description, false);
249  }
250
251  /**
252   * Create a mutable rate metric (for throughput measurement)
253   * @param name  of the metric
254   * @param desc  description
255   * @param extended  produce extended stat (stdev/min/max etc.) if true
256   * @return a new mutable rate metric object
257   */
258  public MutableRate newRate(String name, String desc, boolean extended) {
259    return newRate(name, desc, extended, true);
260  }
261
262  @InterfaceAudience.Private
263  public synchronized MutableRate newRate(String name, String desc,
264      boolean extended, boolean returnExisting) {
265    if (returnExisting) {
266      MutableMetric rate = metricsMap.get(name);
267      if (rate != null) {
268        if (rate instanceof MutableRate) return (MutableRate) rate;
269        throw new MetricsException("Unexpected metrics type "+ rate.getClass()
270                                   +" for "+ name);
271      }
272    }
273    checkMetricName(name);
274    MutableRate ret = new MutableRate(name, desc, extended);
275    metricsMap.put(name, ret);
276    return ret;
277  }
278
279  public synchronized MutableRatesWithAggregation newRatesWithAggregation(
280      String name) {
281    checkMetricName(name);
282    MutableRatesWithAggregation rates = new MutableRatesWithAggregation();
283    metricsMap.put(name, rates);
284    return rates;
285  }
286
287  synchronized void add(String name, MutableMetric metric) {
288    checkMetricName(name);
289    metricsMap.put(name, metric);
290  }
291
292  /**
293   * Add sample to a stat metric by name.
294   * @param name  of the metric
295   * @param value of the snapshot to add
296   */
297  public synchronized void add(String name, long value) {
298    MutableMetric m = metricsMap.get(name);
299
300    if (m != null) {
301      if (m instanceof MutableStat) {
302        ((MutableStat) m).add(value);
303      }
304      else {
305        throw new MetricsException("Unsupported add(value) for metric "+ name);
306      }
307    }
308    else {
309      metricsMap.put(name, newRate(name)); // default is a rate metric
310      add(name, value);
311    }
312  }
313
314  /**
315   * Set the metrics context tag
316   * @param name of the context
317   * @return the registry itself as a convenience
318   */
319  public MetricsRegistry setContext(String name) {
320    return tag(MsInfo.Context, name, true);
321  }
322
323  /**
324   * Add a tag to the metrics
325   * @param name  of the tag
326   * @param description of the tag
327   * @param value of the tag
328   * @return the registry (for keep adding tags)
329   */
330  public MetricsRegistry tag(String name, String description, String value) {
331    return tag(name, description, value, false);
332  }
333
334  /**
335   * Add a tag to the metrics
336   * @param name  of the tag
337   * @param description of the tag
338   * @param value of the tag
339   * @param override  existing tag if true
340   * @return the registry (for keep adding tags)
341   */
342  public MetricsRegistry tag(String name, String description, String value,
343                             boolean override) {
344    return tag(Interns.info(name, description), value, override);
345  }
346
347  /**
348   * Add a tag to the metrics
349   * @param info  metadata of the tag
350   * @param value of the tag
351   * @param override existing tag if true
352   * @return the registry (for keep adding tags etc.)
353   */
354  public synchronized
355  MetricsRegistry tag(MetricsInfo info, String value, boolean override) {
356    if (!override) checkTagName(info.name());
357    tagsMap.put(info.name(), Interns.tag(info, value));
358    return this;
359  }
360
361  public MetricsRegistry tag(MetricsInfo info, String value) {
362    return tag(info, value, false);
363  }
364
365  Collection<MetricsTag> tags() {
366    return tagsMap.values();
367  }
368
369  Collection<MutableMetric> metrics() {
370    return metricsMap.values();
371  }
372
373  private void checkMetricName(String name) {
374    // Check for invalid characters in metric name
375    boolean foundWhitespace = false;
376    for (int i = 0; i < name.length(); i++) {
377      char c = name.charAt(i);
378      if (Character.isWhitespace(c)) {
379        foundWhitespace = true;
380        break;
381      }
382    }
383    if (foundWhitespace) {
384      throw new MetricsException("Metric name '"+ name +
385          "' contains illegal whitespace character");
386    }
387    // Check if name has already been registered
388    if (metricsMap.containsKey(name)) {
389      throw new MetricsException("Metric name "+ name +" already exists!");
390    }
391  }
392
393  private void checkTagName(String name) {
394    if (tagsMap.containsKey(name)) {
395      throw new MetricsException("Tag "+ name +" already exists!");
396    }
397  }
398
399  /**
400   * Sample all the mutable metrics and put the snapshot in the builder
401   * @param builder to contain the metrics snapshot
402   * @param all get all the metrics even if the values are not changed.
403   */
404  public synchronized void snapshot(MetricsRecordBuilder builder, boolean all) {
405    for (MetricsTag tag : tags()) {
406      builder.add(tag);
407    }
408    for (MutableMetric metric : metrics()) {
409      metric.snapshot(builder, all);
410    }
411  }
412
413  @Override public String toString() {
414    return Objects.toStringHelper(this)
415        .add("info", metricsInfo).add("tags", tags()).add("metrics", metrics())
416        .toString();
417  }
418}