You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by Apache Wiki <wi...@apache.org> on 2010/05/27 20:09:33 UTC

[Hadoop Wiki] Update of "HADOOP-6728-MetricsV2" by LukeLu

Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Hadoop Wiki" for change notification.

The "HADOOP-6728-MetricsV2" page has been changed by LukeLu.
The comment on this change is: First cut.
http://wiki.apache.org/hadoop/HADOOP-6728-MetricsV2

--------------------------------------------------

New page:
''This page keeps the design notes for [[https://issues.apache.org/jira/browse/HADOOP-6728|HADOOP-6728]] only. Current dev/user documentation for metrics system should be kept elsewhere.''

== Scope ==

 * Allow multiple metrics output plugins to be used correctly in parallel ([[https://issues.apache.org/jira/browse/HADOOP-6508|HADOOP-6508]]).
 * Allow dynamic (without server restart) reconfiguration of metrics plugins
   * Including filtering (glob patterns preferred by ops. cf. [[https://issues.apache.org/jira/browse/HADOOP-6787|HADOOP-6787]]) metrics by names.
 * Allow all metrics to be exported via JMX.
  
While we realize that there are many optimization opportunities, the main goal is to fix existing issues and lay down the foundation for future improvements.

== Design Overview ==

In the new framework, the metrics sources are where the metrics are generated/updated, and metrics sinks consume the records generated by the metrics sources. A metrics system would poll the metrics sources periodically and pass the metrics records to metrics sinks (Figure 1). The source getMetrics interface allows lockless implementations of metrics instrumentation (with volatile metrics values). The sink interface is simple, where the putMetrics method would be called with an immutable metrics record (Figure 2), so that plugin implementers don't have to worry about thread safety. 

||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445679/metrics2-uml.png|Metrics system overview}}||
||<:>Figure 1: Metrics system overview||

Figure 1 is a UML class diagram illustrating the involved passive objects (different colors indicating different kinds of driving threads in discussion): !MetricsSource (in cyan) is driven by a timer thread (for getMetrics()) and !MetricSink (in green) is driven by a thread for each Sink. The !MetricsFilter objects (in orange) can be used either to filter the metrics from sources in the timer thread or to filter metrics per sink in its respective thread. The metrics system expects that the getMetrics call would return fairly quickly (i.e., latency smaller than the polling period). The !MetricsSinkQueue is a nonblocking queue with preconfigured size (tolerance of sink latency: n * period). New metrics records would be lost if the queue is full. The JMX MBean interface would be implemented to allow existing JMX clients (JConsole, jManage etc.) to stop and start the metrics system at run time.

||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445680/metrics2-record.png|Immutable metrics objects}}||
||<:>Figure 2: Immutable metrics objects||

As an example, a !JobTracker metrics instrumentation would implement the !MetricsSource interface and override the getMetrics method to return a snapshot of metrics when the timer thread polls it with a !MetricsBuilder with configured source filters. The timer thread would then enqueue the resulting metrics records from the !MetricsBuilder to each !MetricsSinkQueue. The thread for each sink would blocked/wait on each !MetricsSinkQueue until it's notified by the timer thread after new metrics records are enqueued and then proceeds to dequeue and call the putMetrics method of the corresponding sink object. Figure 3 and 4 illustrate the new data flow vs the old data flow.

||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445685/metrics1-flow.png|Before}}||{{https://issues.apache.org/jira/secure/attachment/12445686/metrics2-flow.png|After}}||
||<:>Figure 3: V1 data flow diagram||<:>Figure 4: V2 data flow diagram||

The following new packages are proposed for a smooth transition of the metrics framework:
||'''package'''||'''Contents'''||'''Notes'''||
||`org.apache.hadoop.metrics.api`||Public interface and reusable components for the metrics system||Allows easy creation of a hadoop-metrics-api-version.jar for plugin writers.||
||`org.apache.hadoop.metrics.impl`||Metrics system implementations||||
||`org.apache.hadoop.metrics.source`||Public (class names) builtin metrics source (JVM etc.) classes||||
||`org.apache.hadoop.metrics.sink`||Public (class names) builtin metrics sink (file, ganglia etc.) classes||||


== Metrics Filtering ==

The new framework supports 3 levels of filters: source, record and metrics names, thus 6 ways to filter metrics with increasing cost (in terms of memory/CPU):

 1. Global source name filtering: any sources with matching names are skipped for getMetrics calls.
 1. Per sink source name filtering: any sources with matching names are skipped for putMetrics calls.
 1. Per source record filtering: any records with matching names are skipped in the Metrics``Builder.add* calls in the getMetrics calls.
 1. Per sink record filtering: any records with matching names are skipped for the putMetrics calls.
 1. Per source metrics filtering: any metrics with matching names are skipped in the Metric.sample* calls in the getMetrics calls.
 1. Per sink metrics filtering: any metrics with matching names are skipped in the iteration of the Metrics``Record in putMetrics calls.

These can be mixed and matched to optimize for lower total filtering cost if necessary. See below for configuration examples.


== Configuration ==

The new framework uses the [[http://commons.apache.org/configuration/apidocs/org/apache/commons/configuration/PropertiesConfiguration.html|PropertiesConfiguration]] from the [[http://commons.apache.org/configuration/|apache commons configuration library]] for backward compatibility (java properties) and more features (include, variable substitution, subset etc.)

Proposed configuration examples:
{{{
# Basic syntax: <prefix>.(source|sink|filter|timer|jmx).<option>
default.filter.class=org.apache.hadoop.metrics.filter.GlobFilter
default.timer.period=10

default.sink.count=2
default.sink.1.class=org.apache.hadoop.metrics.sink.file.FileSink
default.sink.2.class=com.example.hadoop.MonitorSink

# Filter out any source named *Details
mapred.filter.exclude.sources=*Details

# variable substitution works here
jobtracker.filter.exclude.sources=${mapred.filter.exclude.sources}

# Filter out records with names that matches pattern in the source named "rpc"
# source.count is only for override source options only and doesn't affect actual number of sources 
jobtracker.source.count=1
jobtracker.source.1.name=rpc
jobtracker.source.1.filter.exclude.records=foo*

jobtracker.sink.count=2
# Filter out metrics with names that matches pattern for sink 1 only
jobtracker.sink.1.filter.exclude.records=foo*
# Alternative filter class
jobtracker.sink.2.filter.class=org.apache.hadoop.metrics.filter.RegexFilter
jobtracker.sink.2.filter.exclude.metrics=^bar.*
}}}


== Metrics Source (Instrumentation) Development ==

Implementing a simple metrics source:
{{{#!java
public class MyMetrics extends AbstractMetricsSource {
  // Use MetricMutable (Figure 5) for instrumentation 
  public final MetricMutableGauge<Long> foo;
  public final MetricMutableCounter<Long> bar;

  public MyMetricSource(MetricsSystem ms, String name) {
    super(ms, name);
    MetricsRegistry r = getMetricsRegistry();
    foo = r.addGauge("foo", "varying metric for foo", 0L);
    bar = r.addCounter("bar", "counter for bar", 0L);
    register();
  }
}
}}}

||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445681/metrics2-mutable.png|Mutable metrics for instrumentation}}||
||<:>Figure 5: Mutable metrics helper objects for instrumentation||

Using the metrics source:
{{{#!java
// Somewhere in your app's startup code, initialize the metric system.
DefaultMetricsSystem.INSTANCE.init("metrics_config_prefix");

// Create the metrics source object
MyMetrics myMetrics = new MyMetrics(DefaultMetricsSystem.INSTANCE, "foobar");

// Update the metrics
myMetrics.foo.set(someValue);
myMetrics.bar.inc();
}}}

Note, you don't have to override getMetrics in this simple class, as the parent class provides a reasonable getMetrics implementation. You can override the method if you want to create more records on the fly in addition to the default one. Here is the likely implementation of getMetrics using the metrics builder API (Figure 6.):

{{{#!java
public void getMetrics(MetricsBuilder builder) {
  // the default record name is the same as the source name
  MetricsRecordBuilder rb = builder.addRecord(getName());
  
  for (MetricMutable metric : getMetricsRegistry())
    metric.sample(rb);
}
}}}

||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445687/metrics2-builder.png|Metrics builders}}||
||<:>Figure 6: Metrics builders||

== Metrics Sink (Plugin) Developement ==

Implementing a sink plugin with schema conversion (without a forest of if's):
{{{#!java
public class EchoPlugin implements MetricsSink {
  private final MetricVisitor echoVisitor = new MetricVisitor() {
      public void counter(String name, long value) {
        echoCounterInt64(name, value);
      }
      public void counter(String name, float value) {
        echoCounterFP32(name, value);
      }
      public void counter(String name, double value) {
        echoCounterFP64(name, value);
      }
      public void gauge(String name, byte value) {
        echoGaugeInt8(name, value);
      }
      public void gauge(String name, short value) {
        echoGaugeInt16(name, value);
      }
      public void gauge(String name, int value) {
        echoGaugeInt32(name, value);
      }
      public void gauge(String name, long value) {
        echoGaugeInt64(name, value);
      }
      public void gauge(String name, float value) {
        echoGaugeFP32(name, value);
      }
      public void gauge(String name, double value) {
        echoGaugeFP64(name, value);
      }
    };

  public void init(SubsetConfiguration conf) {
    // do plugin specific initialization here
  }

  public void putMetrics(MetricsRecord rec) {
    for (MetricTag tag : rec.getTags())
      echoTag(tag.getName(), tag.getValue());
        
    for (Metric metric : rec.getMetrics())
      metric.accept(echoVisitor);
  }
}
}}}