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 */
018package org.apache.hadoop.metrics2.lib;
019
020import com.google.errorprone.annotations.RestrictedApi;
021import java.util.Collection;
022import java.util.concurrent.ConcurrentMap;
023import org.apache.hadoop.hbase.metrics.Interns;
024import org.apache.hadoop.metrics2.MetricsException;
025import org.apache.hadoop.metrics2.MetricsInfo;
026import org.apache.hadoop.metrics2.MetricsRecordBuilder;
027import org.apache.hadoop.metrics2.MetricsTag;
028import org.apache.hadoop.metrics2.impl.MsInfo;
029import org.apache.yetus.audience.InterfaceAudience;
030
031import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
032import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
033
034/**
035 * An optional metrics registry class for creating and maintaining a collection of MetricsMutables,
036 * making writing metrics source easier. NOTE: this is a copy of
037 * org.apache.hadoop.metrics2.lib.MetricsRegistry with added one feature: metrics can be removed.
038 * When HADOOP-8313 is fixed, usages of this class should be substituted with
039 * org.apache.hadoop.metrics2.lib.MetricsRegistry. This implementation also provides handy methods
040 * for creating metrics dynamically. Another difference is that metricsMap implementation is
041 * substituted with thread-safe map, as we allow dynamic metrics additions/removals.
042 */
043@InterfaceAudience.Private
044public class DynamicMetricsRegistry {
045
046  private final ConcurrentMap<String, MutableMetric> metricsMap = Maps.newConcurrentMap();
047  private final ConcurrentMap<String, MetricsTag> tagsMap = Maps.newConcurrentMap();
048  private final MetricsInfo metricsInfo;
049  private final DefaultMetricsSystemHelper helper = new DefaultMetricsSystemHelper();
050  private final static String[] histogramSuffixes = new String[] { "_num_ops", "_min", "_max",
051    "_median", "_75th_percentile", "_90th_percentile", "_95th_percentile", "_99th_percentile" };
052
053  /**
054   * Construct the registry with a record name
055   * @param name of the record of the metrics
056   */
057  public DynamicMetricsRegistry(String name) {
058    this(Interns.info(name, name));
059  }
060
061  /**
062   * Construct the registry with a metadata object
063   * @param info the info object for the metrics record/group
064   */
065  public DynamicMetricsRegistry(MetricsInfo info) {
066    metricsInfo = info;
067  }
068
069  /** Returns the info object of the metrics registry */
070  public MetricsInfo info() {
071    return metricsInfo;
072  }
073
074  /**
075   * Get a metric by name
076   * @param name of the metric
077   * @return the metric object
078   */
079  public MutableMetric get(String name) {
080    return metricsMap.get(name);
081  }
082
083  /**
084   * Get a tag by name
085   * @param name of the tag
086   * @return the tag object
087   */
088  public MetricsTag getTag(String name) {
089    return tagsMap.get(name);
090  }
091
092  /**
093   * Create a mutable long integer counter
094   * @param name of the metric
095   * @param desc metric description
096   * @param iVal initial value
097   * @return a new counter object
098   */
099  public MutableFastCounter newCounter(String name, String desc, long iVal) {
100    return newCounter(new MetricsInfoImpl(name, desc), iVal);
101  }
102
103  /**
104   * Create a mutable long integer counter
105   * @param info metadata of the metric
106   * @param iVal initial value
107   * @return a new counter object
108   */
109  public MutableFastCounter newCounter(MetricsInfo info, long iVal) {
110    MutableFastCounter ret = new MutableFastCounter(info, iVal);
111    return addNewMetricIfAbsent(info.name(), ret, MutableFastCounter.class);
112  }
113
114  /**
115   * Create a mutable long integer gauge
116   * @param name of the metric
117   * @param desc metric description
118   * @param iVal initial value
119   * @return a new gauge object
120   */
121  public MutableGaugeLong newGauge(String name, String desc, long iVal) {
122    return newGauge(new MetricsInfoImpl(name, desc), iVal);
123  }
124
125  /**
126   * Create a mutable long integer gauge
127   * @param info metadata of the metric
128   * @param iVal initial value
129   * @return a new gauge object
130   */
131  public MutableGaugeLong newGauge(MetricsInfo info, long iVal) {
132    MutableGaugeLong ret = new MutableGaugeLong(info, iVal);
133    return addNewMetricIfAbsent(info.name(), ret, MutableGaugeLong.class);
134  }
135
136  /**
137   * Create a mutable metric with stats
138   * @param name       of the metric
139   * @param desc       metric description
140   * @param sampleName of the metric (e.g., "Ops")
141   * @param valueName  of the metric (e.g., "Time" or "Latency")
142   * @param extended   produce extended stat (stdev, min/max etc.) if true.
143   * @return a new mutable stat metric object
144   */
145  public MutableStat newStat(String name, String desc, String sampleName, String valueName,
146    boolean extended) {
147    MutableStat ret = new MutableStat(name, desc, sampleName, valueName, extended);
148    return addNewMetricIfAbsent(name, ret, MutableStat.class);
149  }
150
151  /**
152   * Create a mutable metric with stats
153   * @param name       of the metric
154   * @param desc       metric description
155   * @param sampleName of the metric (e.g., "Ops")
156   * @param valueName  of the metric (e.g., "Time" or "Latency")
157   * @return a new mutable metric object
158   */
159  public MutableStat newStat(String name, String desc, String sampleName, String valueName) {
160    return newStat(name, desc, sampleName, valueName, false);
161  }
162
163  /**
164   * Create a mutable rate metric
165   * @param name of the metric
166   * @return a new mutable metric object
167   */
168  public MutableRate newRate(String name) {
169    return newRate(name, name, false);
170  }
171
172  /**
173   * Create a mutable rate metric
174   * @param name        of the metric
175   * @param description of the metric
176   * @return a new mutable rate metric object
177   */
178  public MutableRate newRate(String name, String description) {
179    return newRate(name, description, false);
180  }
181
182  /**
183   * Create a mutable rate metric (for throughput measurement)
184   * @param name     of the metric
185   * @param desc     description
186   * @param extended produce extended stat (stdev/min/max etc.) if true
187   * @return a new mutable rate metric object
188   */
189  public MutableRate newRate(String name, String desc, boolean extended) {
190    return newRate(name, desc, extended, true);
191  }
192
193  @InterfaceAudience.Private
194  public MutableRate newRate(String name, String desc, boolean extended, boolean returnExisting) {
195    if (returnExisting) {
196      MutableMetric rate = metricsMap.get(name);
197      if (rate != null) {
198        if (rate instanceof MutableRate) {
199          return (MutableRate) rate;
200        }
201
202        throw new MetricsException("Unexpected metrics type " + rate.getClass() + " for " + name);
203      }
204    }
205    MutableRate ret = new MutableRate(name, desc, extended);
206    return addNewMetricIfAbsent(name, ret, MutableRate.class);
207  }
208
209  /**
210   * Create a new histogram.
211   * @param name Name of the histogram.
212   * @return A new MutableHistogram
213   */
214  public MutableHistogram newHistogram(String name) {
215    return newHistogram(name, "");
216  }
217
218  /**
219   * Create a new histogram.
220   * @param name The name of the histogram
221   * @param desc The description of the data in the histogram.
222   * @return A new MutableHistogram
223   */
224  public MutableHistogram newHistogram(String name, String desc) {
225    MutableHistogram histo = new MutableHistogram(name, desc);
226    return addNewMetricIfAbsent(name, histo, MutableHistogram.class);
227  }
228
229  /**
230   * Create a new histogram with time range counts.
231   * @param name Name of the histogram.
232   * @return A new MutableTimeHistogram
233   */
234  public MutableTimeHistogram newTimeHistogram(String name) {
235    return newTimeHistogram(name, "");
236  }
237
238  /**
239   * Create a new histogram with time range counts.
240   * @param name The name of the histogram
241   * @param desc The description of the data in the histogram.
242   * @return A new MutableTimeHistogram
243   */
244  public MutableTimeHistogram newTimeHistogram(String name, String desc) {
245    MutableTimeHistogram histo = new MutableTimeHistogram(name, desc);
246    return addNewMetricIfAbsent(name, histo, MutableTimeHistogram.class);
247  }
248
249  /**
250   * Create a new histogram with size range counts.
251   * @param name Name of the histogram.
252   * @return A new MutableSizeHistogram
253   */
254  public MutableSizeHistogram newSizeHistogram(String name) {
255    return newSizeHistogram(name, "");
256  }
257
258  /**
259   * Create a new histogram with size range counts.
260   * @param name The name of the histogram
261   * @param desc The description of the data in the histogram.
262   * @return A new MutableSizeHistogram
263   */
264  public MutableSizeHistogram newSizeHistogram(String name, String desc) {
265    MutableSizeHistogram histo = new MutableSizeHistogram(name, desc);
266    return addNewMetricIfAbsent(name, histo, MutableSizeHistogram.class);
267  }
268
269  synchronized void add(String name, MutableMetric metric) {
270    addNewMetricIfAbsent(name, metric, MutableMetric.class);
271  }
272
273  /**
274   * Add sample to a stat metric by name.
275   * @param name  of the metric
276   * @param value of the snapshot to add
277   */
278  public void add(String name, long value) {
279    MutableMetric m = metricsMap.get(name);
280
281    if (m != null) {
282      if (m instanceof MutableStat) {
283        ((MutableStat) m).add(value);
284      } else {
285        throw new MetricsException("Unsupported add(value) for metric " + name);
286      }
287    } else {
288      metricsMap.put(name, newRate(name)); // default is a rate metric
289      add(name, value);
290    }
291  }
292
293  /**
294   * Set the metrics context tag
295   * @param name of the context
296   * @return the registry itself as a convenience
297   */
298  public DynamicMetricsRegistry setContext(String name) {
299    return tag(MsInfo.Context, name, true);
300  }
301
302  /**
303   * Add a tag to the metrics
304   * @param name        of the tag
305   * @param description of the tag
306   * @param value       of the tag
307   * @return the registry (for keep adding tags)
308   */
309  public DynamicMetricsRegistry tag(String name, String description, String value) {
310    return tag(name, description, value, false);
311  }
312
313  /**
314   * Add a tag to the metrics
315   * @param name        of the tag
316   * @param description of the tag
317   * @param value       of the tag
318   * @param override    existing tag if true
319   * @return the registry (for keep adding tags)
320   */
321  public DynamicMetricsRegistry tag(String name, String description, String value,
322    boolean override) {
323    return tag(new MetricsInfoImpl(name, description), value, override);
324  }
325
326  /**
327   * Add a tag to the metrics
328   * @param info     metadata of the tag
329   * @param value    of the tag
330   * @param override existing tag if true
331   * @return the registry (for keep adding tags etc.)
332   */
333  public DynamicMetricsRegistry tag(MetricsInfo info, String value, boolean override) {
334    MetricsTag tag = Interns.tag(info, value);
335
336    if (!override) {
337      MetricsTag existing = tagsMap.putIfAbsent(info.name(), tag);
338      if (existing != null) {
339        throw new MetricsException("Tag " + info.name() + " already exists!");
340      }
341      return this;
342    }
343
344    tagsMap.put(info.name(), tag);
345
346    return this;
347  }
348
349  public DynamicMetricsRegistry tag(MetricsInfo info, String value) {
350    return tag(info, value, false);
351  }
352
353  Collection<MetricsTag> tags() {
354    return tagsMap.values();
355  }
356
357  Collection<MutableMetric> metrics() {
358    return metricsMap.values();
359  }
360
361  /**
362   * Sample all the mutable metrics and put the snapshot in the builder
363   * @param builder to contain the metrics snapshot
364   * @param all     get all the metrics even if the values are not changed.
365   */
366  public void snapshot(MetricsRecordBuilder builder, boolean all) {
367    for (MetricsTag tag : tags()) {
368      builder.add(tag);
369    }
370    for (MutableMetric metric : metrics()) {
371      metric.snapshot(builder, all);
372    }
373  }
374
375  @Override
376  public String toString() {
377    return MoreObjects.toStringHelper(this).add("info", metricsInfo).add("tags", tags())
378      .add("metrics", metrics()).toString();
379  }
380
381  /**
382   * Removes metric by name
383   * @param name name of the metric to remove
384   */
385  public void removeMetric(String name) {
386    helper.removeObjectName(name);
387    metricsMap.remove(name);
388  }
389
390  public void removeHistogramMetrics(String baseName) {
391    for (String suffix : histogramSuffixes) {
392      helper.removeObjectName(baseName + suffix);
393    }
394    metricsMap.remove(baseName);
395  }
396
397  /**
398   * Get a MetricMutableGaugeLong from the storage. If it is not there atomically put it.
399   * @param gaugeName              name of the gauge to create or get.
400   * @param potentialStartingValue value of the new gauge if we have to create it.
401   */
402  public MutableGaugeLong getGauge(String gaugeName, long potentialStartingValue) {
403    // Try and get the guage.
404    MutableMetric metric = metricsMap.get(gaugeName);
405
406    // If it's not there then try and put a new one in the storage.
407    if (metric == null) {
408
409      // Create the potential new gauge.
410      MutableGaugeLong newGauge =
411        new MutableGaugeLong(new MetricsInfoImpl(gaugeName, ""), potentialStartingValue);
412
413      // Try and put the gauge in. This is atomic.
414      metric = metricsMap.putIfAbsent(gaugeName, newGauge);
415
416      // If the value we get back is null then the put was successful and we will return that.
417      // otherwise gaugeLong should contain the thing that was in before the put could be completed.
418      if (metric == null) {
419        return newGauge;
420      }
421    }
422
423    if (!(metric instanceof MutableGaugeLong)) {
424      throw new MetricsException("Metric already exists in registry for metric name: " + gaugeName
425        + " and not of type MetricMutableGaugeLong");
426    }
427
428    return (MutableGaugeLong) metric;
429  }
430
431  /**
432   * Get a MetricMutableGaugeInt from the storage. If it is not there atomically put it.
433   * @param gaugeName              name of the gauge to create or get.
434   * @param potentialStartingValue value of the new gauge if we have to create it.
435   */
436  public MutableGaugeInt getGaugeInt(String gaugeName, int potentialStartingValue) {
437    // Try and get the guage.
438    MutableMetric metric = metricsMap.get(gaugeName);
439
440    // If it's not there then try and put a new one in the storage.
441    if (metric == null) {
442      // Create the potential new gauge.
443      MutableGaugeInt newGauge =
444        new MutableGaugeInt(new MetricsInfoImpl(gaugeName, ""), potentialStartingValue);
445
446      // Try and put the gauge in. This is atomic.
447      metric = metricsMap.putIfAbsent(gaugeName, newGauge);
448
449      // If the value we get back is null then the put was successful and we will return that.
450      // otherwise gaugeInt should contain the thing that was in before the put could be completed.
451      if (metric == null) {
452        return newGauge;
453      }
454    }
455
456    if (!(metric instanceof MutableGaugeInt)) {
457      throw new MetricsException("Metric already exists in registry for metric name: " + gaugeName
458        + " and not of type MetricMutableGaugeInr");
459    }
460
461    return (MutableGaugeInt) metric;
462  }
463
464  /**
465   * Get a MetricMutableCounterLong from the storage. If it is not there atomically put it.
466   * @param counterName            Name of the counter to get
467   * @param potentialStartingValue starting value if we have to create a new counter
468   */
469  public MutableFastCounter getCounter(String counterName, long potentialStartingValue) {
470    // See getGauge for description on how this works.
471    MutableMetric counter = metricsMap.get(counterName);
472    if (counter == null) {
473      MutableFastCounter newCounter =
474        new MutableFastCounter(new MetricsInfoImpl(counterName, ""), potentialStartingValue);
475      counter = metricsMap.putIfAbsent(counterName, newCounter);
476      if (counter == null) {
477        return newCounter;
478      }
479    }
480
481    if (!(counter instanceof MutableCounter)) {
482      throw new MetricsException("Metric already exists in registry for metric name: " + counterName
483        + " and not of type MutableCounter");
484    }
485
486    return (MutableFastCounter) counter;
487  }
488
489  public MutableHistogram getHistogram(String histoName) {
490    // See getGauge for description on how this works.
491    MutableMetric histo = metricsMap.get(histoName);
492    if (histo == null) {
493      MutableHistogram newCounter = new MutableHistogram(new MetricsInfoImpl(histoName, ""));
494      histo = metricsMap.putIfAbsent(histoName, newCounter);
495      if (histo == null) {
496        return newCounter;
497      }
498    }
499
500    if (!(histo instanceof MutableHistogram)) {
501      throw new MetricsException("Metric already exists in registry for metric name: " + histoName
502        + " and not of type MutableHistogram");
503    }
504
505    return (MutableHistogram) histo;
506  }
507
508  private <T extends MutableMetric> T addNewMetricIfAbsent(String name, T ret,
509    Class<T> metricClass) {
510    // If the value we get back is null then the put was successful and we will
511    // return that. Otherwise metric should contain the thing that was in
512    // before the put could be completed.
513    MutableMetric metric = metricsMap.putIfAbsent(name, ret);
514    if (metric == null) {
515      return ret;
516    }
517
518    return returnExistingWithCast(metric, metricClass, name);
519  }
520
521  @SuppressWarnings("unchecked")
522  private <T> T returnExistingWithCast(MutableMetric metric, Class<T> metricClass, String name) {
523    if (!metricClass.isAssignableFrom(metric.getClass())) {
524      throw new MetricsException("Metric already exists in registry for metric name: " + name
525        + " and not of type " + metricClass + " but instead of type " + metric.getClass());
526    }
527
528    return (T) metric;
529  }
530
531  public void clearMetrics() {
532    for (String name : metricsMap.keySet()) {
533      helper.removeObjectName(name);
534    }
535    metricsMap.clear();
536  }
537
538  @RestrictedApi(explanation = "Should only be called in TestMetricsTableMetricsMap", link = "",
539      allowedOnPath = ".*/(DynamicMetricsRegistry|TestMetricsTableMetricsMap).java")
540  public ConcurrentMap<String, MutableMetric> getMetricsMap() {
541    return metricsMap;
542  }
543}