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);
+   }
+ }
+ }}}
+ 

Reply via email to