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    
019    package org.apache.hadoop.metrics2.lib;
020    
021    import java.util.Collection;
022    import java.util.Map;
023    
024    import com.google.common.collect.Maps;
025    import com.google.common.base.Objects;
026    
027    import org.apache.hadoop.classification.InterfaceAudience;
028    import org.apache.hadoop.classification.InterfaceStability;
029    import org.apache.hadoop.metrics2.MetricsInfo;
030    import org.apache.hadoop.metrics2.MetricsException;
031    import org.apache.hadoop.metrics2.MetricsRecordBuilder;
032    import org.apache.hadoop.metrics2.MetricsTag;
033    import 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
041    public 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      synchronized void add(String name, MutableMetric metric) {
280        checkMetricName(name);
281        metricsMap.put(name, metric);
282      }
283    
284      /**
285       * Add sample to a stat metric by name.
286       * @param name  of the metric
287       * @param value of the snapshot to add
288       */
289      public synchronized void add(String name, long value) {
290        MutableMetric m = metricsMap.get(name);
291    
292        if (m != null) {
293          if (m instanceof MutableStat) {
294            ((MutableStat) m).add(value);
295          }
296          else {
297            throw new MetricsException("Unsupported add(value) for metric "+ name);
298          }
299        }
300        else {
301          metricsMap.put(name, newRate(name)); // default is a rate metric
302          add(name, value);
303        }
304      }
305    
306      /**
307       * Set the metrics context tag
308       * @param name of the context
309       * @return the registry itself as a convenience
310       */
311      public MetricsRegistry setContext(String name) {
312        return tag(MsInfo.Context, name, true);
313      }
314    
315      /**
316       * Add a tag to the metrics
317       * @param name  of the tag
318       * @param description of the tag
319       * @param value of the tag
320       * @return the registry (for keep adding tags)
321       */
322      public MetricsRegistry tag(String name, String description, String value) {
323        return tag(name, description, value, false);
324      }
325    
326      /**
327       * Add a tag to the metrics
328       * @param name  of the tag
329       * @param description of the tag
330       * @param value of the tag
331       * @param override  existing tag if true
332       * @return the registry (for keep adding tags)
333       */
334      public MetricsRegistry tag(String name, String description, String value,
335                                 boolean override) {
336        return tag(Interns.info(name, description), value, override);
337      }
338    
339      /**
340       * Add a tag to the metrics
341       * @param info  metadata of the tag
342       * @param value of the tag
343       * @param override existing tag if true
344       * @return the registry (for keep adding tags etc.)
345       */
346      public synchronized
347      MetricsRegistry tag(MetricsInfo info, String value, boolean override) {
348        if (!override) checkTagName(info.name());
349        tagsMap.put(info.name(), Interns.tag(info, value));
350        return this;
351      }
352    
353      public MetricsRegistry tag(MetricsInfo info, String value) {
354        return tag(info, value, false);
355      }
356    
357      Collection<MetricsTag> tags() {
358        return tagsMap.values();
359      }
360    
361      Collection<MutableMetric> metrics() {
362        return metricsMap.values();
363      }
364    
365      private void checkMetricName(String name) {
366        if (metricsMap.containsKey(name)) {
367          throw new MetricsException("Metric name "+ name +" already exists!");
368        }
369      }
370    
371      private void checkTagName(String name) {
372        if (tagsMap.containsKey(name)) {
373          throw new MetricsException("Tag "+ name +" already exists!");
374        }
375      }
376    
377      /**
378       * Sample all the mutable metrics and put the snapshot in the builder
379       * @param builder to contain the metrics snapshot
380       * @param all get all the metrics even if the values are not changed.
381       */
382      public synchronized void snapshot(MetricsRecordBuilder builder, boolean all) {
383        for (MetricsTag tag : tags()) {
384          builder.add(tag);
385        }
386        for (MutableMetric metric : metrics()) {
387          metric.snapshot(builder, all);
388        }
389      }
390    
391      @Override public String toString() {
392        return Objects.toStringHelper(this)
393            .add("info", metricsInfo).add("tags", tags()).add("metrics", metrics())
394            .toString();
395      }
396    }