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/08/21 03:45:09 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.
http://wiki.apache.org/hadoop/HADOOP-6728-MetricsV2?action=diff&rev1=1&rev2=2

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

  
  == 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. 
+ In the v2 framework, 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}}||
+ ||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12452696/metrics2-uml-r2.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.
+ Figure 1 is a [[http://martinfowler.com/bliki/UmlAsSketch.html|UML sketch]] 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}}||
+ ||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12452693/metrics2-record-r2.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.
+ 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. 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.metrics2`||Public interface/abstract classes of the metrics system||||
+ ||`org.apache.hadoop.metrics2.annotation`||Public annotations for implementing simple metrics sources||||
+ ||`org.apache.hadoop.metrics2.lib`||Public reusable components for using the metrics system||||
+ ||`org.apache.hadoop.metrics2.filter`||Public (class names) builtin metrics filter (Glob/Regex) classes||||
- ||`org.apache.hadoop.metrics.source`||Public (class names) builtin metrics source (JVM etc.) classes||||
+ ||`org.apache.hadoop.metrics2.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||||
+ ||`org.apache.hadoop.metrics2.sink`||Public (class names) builtin metrics sink (file, ganglia etc.) classes||||
- 
+ ||`org.apache.hadoop.metrics2.util`||Public utility classes for developing metrics system (including plugins)||||
+ ||`org.apache.hadoop.metrics2.impl`||Metrics system internal implementation 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):
+ The 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 source record filtering: any records with matching names or tag values 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 sink record filtering: any records with matching names or tag values 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>
+ # Basic syntax: <prefix>.(source|sink).<instance>.<option>
+ *.sink.file.class=org.apache.hadoop.metrics2.sink.FileSink
- default.filter.class=org.apache.hadoop.metrics.filter.GlobFilter
+ *.source.filter.class=org.apache.hadoop.metrics2.filter.GlobFilter
- default.timer.period=10
+ *.record.filter.class=${*.source.filter.class}
+ *.metric.filter.class=${*.source.filter.class}
+ *.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
+ # Filter out any sources with names end with Details
- mapred.filter.exclude.sources=*Details
+ jobtracker.*.source.filter.exclude=*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"
+ # Filter out records with names that matches foo* 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.source.rpc.record.filter.exclude=foo*
  
- jobtracker.sink.count=2
- # Filter out metrics with names that matches pattern for sink 1 only
+ # Filter out metrics with names that matches foo* for sink instance "file" only
- jobtracker.sink.1.filter.exclude.records=foo*
+ jobtracker.sink.file.metric.filter.exclude=foo*
+ jobtracker.sink.file.filename=jt-metrics.out
- # Alternative filter class
- jobtracker.sink.2.filter.class=org.apache.hadoop.metrics.filter.RegexFilter
- jobtracker.sink.2.filter.exclude.metrics=^bar.*
- }}}
  
+ # Custom sink plugin
+ jobtracker.sink.my.class=com.example.hadoop.metrics.my.MyPlugin
+ # MyPlugin only handles metrics in "foo" context
+ jobtracker.sink.my.context=foo
+ }}}
  
  == Metrics Source (Instrumentation) Development ==
  
- Implementing a simple metrics source:
+ A minimal metrics source:
  {{{#!java
- public class MyMetrics extends AbstractMetricsSource {
-   // Use MetricMutable (Figure 5) for instrumentation 
-   public final MetricMutableGauge<Long> foo;
-   public final MetricMutableCounter<Long> bar;
+ // default record name is the class name
+ // default context name is "default"
+ @Metrics(context="bar")
+ public class MyPojo {
+   // Default name of metric is method name sans get
+   // Default type of metric is gauge
+   @Metric("An integer gauge named MyMetric")
+   public int getMyMetric() { return 42; }
  
+   // Recommended helper method
+   public MyMetric registerWith(MetricsSystem ms) {
+     return ms.register("MyPojo", "MyPojo metrics", this);
-   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();
    }
  }
  }}}
  
+ By using annotations, one can add simple metrics to any methods returning supported types (int, long, float and double and their object counter parts) in any java classes.
+ 
+ An example using the mutable metric library objects:
+ {{{#!java
+ @Metrics{context="bar")
+ public class MyMetrics {
+   // Default metric name is the variable name
+   @Metric("An integer gauge") MutableGaugeInt g1;
+   // Default type is inferred from the mutable metric type
+   @Metric("An long integer counter") MutableCounterLong c1;
+ 
+   // Recommended helper method
+   public MyMetrics registerWith(MetricsSystem ms) {
+     return ms.register("MyMetrics2", "MyMetrics2 description", this);
+   }
+ }
+ }}}
+ 
- ||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445681/metrics2-mutable.png|Mutable metrics for instrumentation}}||
+ ||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12452694/metrics2-mutable-r2.png|Mutable metrics for instrumentation}}||
  ||<:>Figure 5: Mutable metrics helper objects for instrumentation||
  
- Using the metrics source:
+ Initialize the metrics system:
  {{{#!java
  // Somewhere in your app's startup code, initialize the metric system.
- DefaultMetricsSystem.INSTANCE.init("metrics_config_prefix");
+ DefaultMetricsSystem.initialize("jobtracker");
  
  // Create the metrics source object
- MyMetrics myMetrics = new MyMetrics(DefaultMetricsSystem.INSTANCE, "foobar");
+ MyMetrics myMetrics = new MyMetrics().registerWith(DefaultMetricsSystem.INSTANCE);
  
  // Update the metrics
  myMetrics.foo.set(someValue);
- myMetrics.bar.inc();
+ myMetrics.bar.incr();
  }}}
  
- 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.):
+ Note, for simple metrics sources, using annotations make things declarative and concise. For more advanced metrics source implementations, you might need to explicitly implement the MetricsSource interface and override the getMetrics method and use the metrics builder API (Figure 6.):
  
  {{{#!java
+ class MyMetricsSource implements MetricsSource {
+ 
+   @Override
- public void getMetrics(MetricsBuilder builder) {
+   public void getMetrics(MetricsBuilder builder) {
-   // the default record name is the same as the source name
-   MetricsRecordBuilder rb = builder.addRecord(getName());
+     builder.addRecord("foo")
+       .addGauge("g0", "an integer gauge", 42)
+       .addCounter("c0", "a long counter", 42L);
+ 
+     // Typical metrics sources generate one record per snapshot.
+     // We can add more records, which is not supported by annotations.
+     builder.addRecord("bar")
+       .addGauge("g1", "a float gauge", 42.0)
+       .addCounter("c1", "a integer counter", 42);
-   
+   }
-   for (MetricMutable metric : getMetricsRegistry())
-     metric.sample(rb);
- }
- }}}
  
+   public MyMetricSource registerWith(MetricsSystem ms) {
+     return ms.register("MyMetrics", "MyMetrics description", this);
+   }
+ }
+ }}}
+ 
- ||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12445687/metrics2-builder.png|Metrics builders}}||
+ ||<tablewidth="99%">{{https://issues.apache.org/jira/secure/attachment/12452695/metrics2-builder-r2.png|Metrics builders}}||
  ||<:>Figure 6: Metrics builders||
  
  == Metrics Sink (Plugin) Developement ==
  
- Implementing a sink plugin with schema conversion (without a forest of if's):
+ Implementing a sink plugin with schema conversion (without a forest of if/switches):
  {{{#!java
- public class EchoPlugin implements MetricsSink {
+ public class EchoPlugin implements MetricsSink, MetricsVisitor {
-   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);
-       }
-     };
  
+   @Override // MetricsPlugin
    public void init(SubsetConfiguration conf) {
      // do plugin specific initialization here
    }
  
+   @Override // MetricsSink
    public void putMetrics(MetricsRecord rec) {
+     echoHeader(rec.name(), rec.context());
+ 
-     for (MetricTag tag : rec.getTags())
+     for (MetricTag tag : rec.tags())
        echoTag(tag.getName(), tag.getValue());
-         
+   
-     for (Metric metric : rec.getMetrics())
+     for (Metric metric : rec.metrics())
-       metric.accept(echoVisitor);
+       metric.visit(this);
    }
- }
- }}}
  
+   @Override // MetricsSink
+   public void flush() {
+     // do sink specific buffer management here
+   }
+ 
+   @Override // MetricsVisitor
+   public void counter(MetricInfo info, int value) {
+     echoCounterInt32(info.name(), value);
+   }
+ 
+   @Override // MetricsVisitor
+   public void counter(MetricInfo info, long value) {
+     echoCounterInt64(info.name(), value);
+   }
+ 
+   @Override // MetricsVisitor
+   public void gauge(MetricInfo info, int value) {
+     echoGaugeInt32(info.name(), value);
+   }
+ 
+   @Override // MetricsVisitor
+   public void gauge(MetricInfo info, long value) {
+     echoGaugeInt64(info.name(), value);
+   }
+ 
+   @Override // MetricsVisitor
+   public void gauge(MetricInfo info, float value) {
+     echoGaugeFp32(info.name(), value);
+   }
+ 
+   @Override // MetricsVisitor
+   public void gauge(MetricInfo info, double value) {
+     echoGaugeFp64(info.name(), value);
+   }
+ }
+ }}}
+