@InterfaceAudience.Public @InterfaceStability.Evolving

Package org.apache.hadoop.metrics2

Metrics 2.0

See:
          Description

Interface Summary
MetricsCollector The metrics collector interface
MetricsInfo Interface to provide immutable meta info for metrics
MetricsPlugin The plugin interface for the metrics framework
MetricsRecord An immutable snapshot of metrics with a timestamp
MetricsSink The metrics sink interface
MetricsSource The metrics source interface
MetricsSystemMXBean The JMX interface to the metrics system
MetricsVisitor A visitor interface for metrics
 

Class Summary
AbstractMetric The immutable metric
MetricsFilter The metrics filter interface
MetricsRecordBuilder The metrics record builder interface
MetricsSystem The metrics system interface
MetricsTag Immutable tag for metrics (for grouping on host/queue/username etc.)
 

Exception Summary
MetricsException A general metrics exception wrapper
 

Package org.apache.hadoop.metrics2 Description

Metrics 2.0

Overview

This package provides a framework for metrics instrumentation and publication.

The framework provides a variety of ways to implement metrics instrumentation easily via the simple MetricsSource interface or the even simpler and more concise and declarative metrics annotations. The consumers of metrics just need to implement the simple MetricsSink interface. Producers register the metrics sources with a metrics system, while consumers register the sinks. A default metrics system is provided to marshal metrics from sources to sinks based on (per source/sink) configuration options. All the metrics are also published and queryable via the standard JMX MBean interface. This document targets the framework users. Framework developers could also consult the design document for architecture and implementation notes.

Sub-packages

org.apache.hadoop.metrics2.annotation
Public annotation interfaces for simpler metrics instrumentation.
org.apache.hadoop.metrics2.impl
Implementation classes of the framework for interface and/or abstract classes defined in the top-level package. Sink plugin code usually does not need to reference any class here.
org.apache.hadoop.metrics2.lib
Convenience classes for implementing metrics sources, including the Mutable[Gauge*| Counter*| Stat] and MetricsRegistry.
org.apache.hadoop.metrics2.filter
Builtin metrics filter implementations include the GlobFilter and RegexFilter.
org.apache.hadoop.metrics2.source
Builtin metrics source implementations including the JvmMetrics.
org.apache.hadoop.metrics2.sink
Builtin metrics sink implementations including the FileSink.
org.apache.hadoop.metrics2.util
General utilities for implementing metrics sinks etc., including the MetricsCache.

Getting started

Implementing metrics sources

Using annotationsUsing MetricsSource interface
  @Metrics(context="MyContext")
  class MyStat {

    @Metric("My metric description")
    public int getMyMetric() {
      return 42;
    }
  }
  class MyStat implements MetricsSource {

    @Override
    public void getMetrics(MetricsCollector collector, boolean all) {
      collector.addRecord("MyStat")
          .setContext("MyContext")
          .addGauge(info("MyMetric", "My metric description"), 42);
    }
  }
  

In this example we introduced the following:

@Metrics
The Metrics annotation is used to indicate that the class is a metrics source.
MyContext
The optional context name typically identifies either the application, or a group of modules within an application or library.
MyStat
The class name is used (by default, or specified by name=value parameter in the Metrics annotation) as the metrics record name for which a set of metrics are to be reported. For example, you could have a record named "CacheStat" for reporting a number of statistics relating to the usage of some cache in your application.
@Metric
The Metric annotation identifies a particular metric, which in this case, is the result of the method call getMyMetric of the "gauge" (default) type, which means it can vary in both directions, compared with a "counter" type, which can only increase or stay the same. The name of the metric is "MyMetric" (inferred from getMyMetric method name by default.) The 42 here is the value of the metric which can be substituted with any valid java expressions.

Note, the MetricsSource interface is more verbose but more flexible, allowing generated metrics names and multiple records. In fact, the annotation interface is implemented with the MetricsSource interface internally.

Implementing metrics sinks

  public class MySink implements MetricsSink {
    public void putMetrics(MetricsRecord record) {
      System.out.print(record);
    }
    public void init(SubsetConfiguration conf) {}
    public void flush() {}
  }

In this example there are three additional concepts:

record
This object corresponds to the record created in metrics sources e.g., the "MyStat" in previous example.
conf
The configuration object for the sink instance with prefix removed. So you can get any sink specific configuration using the usual get* method.
flush
This method is called for each update cycle, which may involve more than one record. The sink should try to flush any buffered metrics to its backend upon the call. But it's not required that the implementation is synchronous.

In order to make use our MyMetrics and MySink, they need to be hooked up to a metrics system. In this case (and most cases), the DefaultMetricsSystem would suffice.

  DefaultMetricsSystem.initialize("test"); // called once per application
  DefaultMetricsSystem.register(new MyStat());

Metrics system configuration

Sinks are usually specified in a configuration file, say, "hadoop-metrics2-test.properties", as:

  test.sink.mysink0.class=com.example.hadoop.metrics.MySink

The configuration syntax is:

  [prefix].[source|sink|jmx|].[instance].[option]

In the previous example, test is the prefix and mysink0 is an instance name. DefaultMetricsSystem would try to load hadoop-metrics2-[prefix].properties first, and if not found, try the default hadoop-metrics2.properties in the class path. Note, the [instance] is an arbitrary name to uniquely identify a particular sink instance. The asterisk (*) can be used to specify default options.

Consult the metrics instrumentation in jvm, rpc, hdfs and mapred, etc. for more examples.

Metrics Filtering

One of the features of the default metrics system is metrics filtering configuration by source, context, record/tags and metrics. The least expensive way to filter out metrics would be at the source level, e.g., filtering out source named "MyMetrics". The most expensive way would be per metric filtering.

Here are some examples:

  test.sink.file0.class=org.apache.hadoop.metrics2.sink.FileSink
  test.sink.file0.context=foo

In this example, we configured one sink instance that would accept metrics from context foo only.

.source.filter.class=org.apache.hadoop.metrics2.filter.GlobFilter
  test.*.source.filter.include=foo
  test.*.source.filter.exclude=bar

In this example, we specify a source filter that includes source foo and excludes bar. When only include patterns are specified, the filter operates in the white listing mode, where only matched sources are included. Likewise, when only exclude patterns are specified, only matched sources are excluded. Sources that are not matched in either patterns are included as well when both patterns are present. Note, the include patterns have precedence over the exclude patterns.

Similarly, you can specify the record.filter and metric.filter options, which operate at record and metric level, respectively. Filters can be combined to optimize the filtering efficiency.

Metrics instrumentation strategy

In previous examples, we showed a minimal example to use the metrics framework. In a larger system (like Hadoop) that allows custom metrics instrumentation, we recommend the following strategy:
  @Metrics(about="My metrics description", context="MyContext")
  class MyMetrics extends MyInstrumentation {

    @Metric("My gauge description") MutableGaugeInt gauge0;
    @Metric("My counter description") MutableCounterLong counter0;
    @Metric("My rate description") MutableRate rate0;

    @Override public void setGauge0(int value) { gauge0.set(value); }
    @Override public void incrCounter0() { counter0.incr(); }
    @Override public void addRate0(long elapsed) { rate0.add(elapsed); }
  }
  
Note, in this example we introduced the following:
MyInstrumentation
This is usually an abstract class (or interface) to define an instrumentation interface (incrCounter0 etc.) that allows different implementations. This could be a mechanism to allow different metrics systems to be used at runtime via configuration.
Mutable[Gauge*|Counter*|Rate]
These are library classes to manage mutable metrics for implementations of metrics sources. They produce immutable gauge and counters (Metric[Gauge*|Counter*]) for downstream consumption (sinks) upon snapshot. The MutableRate in particular, provides a way to measure latency and throughput of an operation. In this particular case, it produces a long counter "Rate0NumOps" and double gauge "Rate0AvgTime" when snapshotted.

Migration from previous system

Users of the previous metrics system would notice the lack of context prefix in the configuration examples. The new metrics system decouples the concept for context (for grouping) with the implementation where a particular context object does the updating and publishing of metrics, which causes problems when you want to have a single context to be consumed by multiple backends. You would also have to configure an implementation instance per context, even if you have a backend that can handle multiple contexts (file, gangalia etc.):

BeforeAfter
  context1.class=org.hadoop.metrics.file.FileContext
  context2.class=org.hadoop.metrics.file.FileContext
  ...
  contextn.class=org.hadoop.metrics.file.FileContext
  myprefix.sink.file.class=org.hadoop.metrics2.sink.FileSink

In the new metrics system, you can simulate the previous behavior by using the context option in the sink options like the following:

BeforeAfter
  context0.class=org.hadoop.metrics.file.FileContext
  context0.fileName=context0.out
  context1.class=org.hadoop.metrics.file.FileContext
  context1.fileName=context1.out
  ...
  contextn.class=org.hadoop.metrics.file.FileContext
  contextn.fileName=contextn.out
  myprefix.sink.*.class=org.apache.hadoop.metrics2.sink.FileSink
  myprefix.sink.file0.context=context0
  myprefix.sink.file0.filename=context1.out
  myprefix.sink.file1.context=context1
  myprefix.sink.file1.filename=context1.out
  ...
  myprefix.sink.filen.context=contextn
  myprefix.sink.filen.filename=contextn.out

to send metrics of a particular context to a particular backend. Note, myprefix is an arbitrary prefix for configuration groupings, typically they are the name of a particular process (namenode, jobtracker, etc.)



Copyright © 2014 Apache Software Foundation. All Rights Reserved.