You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2017/01/25 19:48:45 UTC

[3/3] hbase git commit: HBASE-9774 HBase native metrics and metric collection for coprocessors

HBASE-9774 HBase native metrics and metric collection for coprocessors


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c64a1d19
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c64a1d19
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c64a1d19

Branch: refs/heads/master
Commit: c64a1d199402d6bf2d6ff4168c00c756dcaa59e4
Parents: 59fd6eb
Author: Enis Soztutar <en...@apache.org>
Authored: Wed Jan 25 11:47:35 2017 -0800
Committer: Enis Soztutar <en...@apache.org>
Committed: Wed Jan 25 11:47:35 2017 -0800

----------------------------------------------------------------------
 hbase-assembly/pom.xml                          |  10 +
 .../hadoop/hbase/util/FastLongHistogram.java    | 310 -----------
 .../hbase/util/TestFastLongHistogram.java       | 132 -----
 .../ExampleMasterObserverWithMetrics.java       | 142 +++++
 .../ExampleRegionObserverWithMetrics.java       | 121 +++++
 hbase-hadoop-compat/pom.xml                     |   4 +
 .../apache/hadoop/hbase/metrics/BaseSource.java |   5 +
 hbase-hadoop2-compat/pom.xml                    |   4 +
 .../hadoop/hbase/metrics/BaseSourceImpl.java    |  46 ++
 .../impl/GlobalMetricRegistriesAdapter.java     | 172 ++++++
 .../impl/HBaseMetrics2HadoopMetricsAdapter.java | 169 ++++++
 .../MetricsRegionServerSourceImpl.java          |   6 +
 .../lib/DefaultMetricsSystemHelper.java         |  50 +-
 .../hadoop/metrics2/lib/MutableHistogram.java   | 110 ++--
 .../metrics2/lib/MutableRangeHistogram.java     |  14 +-
 hbase-metrics-api/README.txt                    |  78 +++
 hbase-metrics-api/pom.xml                       | 112 ++++
 .../apache/hadoop/hbase/metrics/Counter.java    |  60 ++
 .../org/apache/hadoop/hbase/metrics/Gauge.java  |  35 ++
 .../apache/hadoop/hbase/metrics/Histogram.java  |  58 ++
 .../org/apache/hadoop/hbase/metrics/Meter.java  |  90 +++
 .../org/apache/hadoop/hbase/metrics/Metric.java |  30 +
 .../hadoop/hbase/metrics/MetricRegistries.java  |  89 +++
 .../hbase/metrics/MetricRegistriesLoader.java   |  96 ++++
 .../hadoop/hbase/metrics/MetricRegistry.java    | 111 ++++
 .../hbase/metrics/MetricRegistryFactory.java    |  36 ++
 .../hbase/metrics/MetricRegistryInfo.java       | 112 ++++
 .../apache/hadoop/hbase/metrics/MetricSet.java  |  41 ++
 .../hadoop/hbase/metrics/PackageMarker.java     |  36 ++
 .../apache/hadoop/hbase/metrics/Snapshot.java   | 136 +++++
 .../org/apache/hadoop/hbase/metrics/Timer.java  |  68 +++
 .../hadoop/hbase/metrics/package-info.java      |  25 +
 .../metrics/TestMetricRegistriesLoader.java     |  56 ++
 hbase-metrics/README.txt                        |   1 +
 hbase-metrics/pom.xml                           | 136 +++++
 .../hadoop/hbase/metrics/impl/CounterImpl.java  |  61 +++
 .../hbase/metrics/impl/DropwizardMeter.java     |  74 +++
 .../hbase/metrics/impl/FastLongHistogram.java   | 397 ++++++++++++++
 .../hbase/metrics/impl/HistogramImpl.java       |  81 +++
 .../metrics/impl/MetricRegistriesImpl.java      |  75 +++
 .../metrics/impl/MetricRegistryFactoryImpl.java |  34 ++
 .../hbase/metrics/impl/MetricRegistryImpl.java  | 122 +++++
 .../hbase/metrics/impl/RefCountingMap.java      |  91 ++++
 .../hadoop/hbase/metrics/impl/TimerImpl.java    |  58 ++
 .../hadoop/hbase/metrics/impl/package-info.java |  25 +
 ...apache.hadoop.hbase.metrics.MetricRegistries |  18 +
 .../hbase/metrics/impl/TestCounterImpl.java     |  59 ++
 .../hbase/metrics/impl/TestDropwizardMeter.java |  51 ++
 .../metrics/impl/TestFastLongHistogram.java     | 132 +++++
 .../hadoop/hbase/metrics/impl/TestGauge.java    |  61 +++
 .../hbase/metrics/impl/TestHistogramImpl.java   | 100 ++++
 .../metrics/impl/TestMetricRegistryImpl.java    | 142 +++++
 .../hbase/metrics/impl/TestRefCountingMap.java  | 154 ++++++
 .../hbase/metrics/impl/TestTimerImpl.java       |  53 ++
 hbase-server/pom.xml                            |   8 +
 .../MasterCoprocessorEnvironment.java           |  10 +
 .../hbase/coprocessor/MetricsCoprocessor.java   | 136 +++++
 .../RegionCoprocessorEnvironment.java           |  24 +-
 .../RegionServerCoprocessorEnvironment.java     |  10 +
 .../coprocessor/WALCoprocessorEnvironment.java  |  10 +
 .../hadoop/hbase/io/hfile/AgeSnapshot.java      |   2 +-
 .../hadoop/hbase/io/hfile/BlockCacheUtil.java   |   2 +-
 .../hadoop/hbase/io/hfile/CacheStats.java       |   2 +-
 .../hbase/master/MasterCoprocessorHost.java     |  27 +-
 .../hbase/regionserver/MetricsRegionServer.java |  21 +-
 .../hbase/regionserver/RSRpcServices.java       |   8 +
 .../regionserver/RegionCoprocessorHost.java     |  21 +-
 .../RegionServerCoprocessorHost.java            |  26 +-
 .../regionserver/wal/WALCoprocessorHost.java    |  16 +
 .../apache/hadoop/hbase/MiniHBaseCluster.java   |   7 +
 .../coprocessor/TestCoprocessorMetrics.java     | 544 +++++++++++++++++++
 .../security/token/TestTokenAuthentication.java |   6 +
 pom.xml                                         |  26 +
 73 files changed, 4848 insertions(+), 547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index b9d8dcc..19bf5f0 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -208,6 +208,16 @@
     </dependency>
     <dependency>
        <groupId>org.apache.hbase</groupId>
+       <artifactId>hbase-metrics-api</artifactId>
+       <version>${project.version}</version>
+    </dependency>
+    <dependency>
+       <groupId>org.apache.hbase</groupId>
+       <artifactId>hbase-metrics</artifactId>
+       <version>${project.version}</version>
+    </dependency>
+    <dependency>
+       <groupId>org.apache.hbase</groupId>
        <artifactId>hbase-resource-bundle</artifactId>
        <version>${project.version}</version>
        <optional>true</optional>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
deleted file mode 100644
index 310348e..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/FastLongHistogram.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.LongAdder;
-import java.util.stream.Stream;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
-
-/**
- * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the
- * quantiles.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class FastLongHistogram {
-
-  /**
-   * Default number of bins.
-   */
-  public static final int DEFAULT_NBINS = 255;
-
-  public static final double[] DEFAULT_QUANTILES =
-      new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
-
-  /**
-   * Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
-   */
-  private static class Bins {
-    private final LongAdder[] counts;
-    // inclusive
-    private final long binsMin;
-    // exclusive
-    private final long binsMax;
-    private final long bins10XMax;
-    private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
-    private final AtomicLong max = new AtomicLong(0L);
-
-    private final LongAdder count = new LongAdder();
-    private final LongAdder total = new LongAdder();
-
-    // set to true when any of data has been inserted to the Bins. It is set after the counts are
-    // updated.
-    private volatile boolean hasData = false;
-
-    /**
-     * The constructor for creating a Bins without any prior data.
-     */
-    public Bins(int numBins) {
-      counts = createCounters(numBins);
-      this.binsMin = 1L;
-
-      // These two numbers are total guesses
-      // and should be treated as highly suspect.
-      this.binsMax = 1000;
-      this.bins10XMax = binsMax * 10;
-    }
-
-    /**
-     * The constructor for creating a Bins with last Bins.
-     */
-    public Bins(Bins last, int numBins, double minQ, double maxQ) {
-      long[] values = last.getQuantiles(new double[] { minQ, maxQ });
-      long wd = values[1] - values[0] + 1;
-      // expand minQ and maxQ in two ends back assuming uniform distribution
-      this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
-      long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
-      // make sure each of bins is at least of width 1
-      this.binsMax = Math.max(binsMax, this.binsMin + numBins);
-      this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
-
-      this.counts = createCounters(numBins);
-    }
-
-    private LongAdder[] createCounters(int numBins) {
-      return Stream.generate(LongAdder::new).limit(numBins + 3).toArray(LongAdder[]::new);
-    }
-
-    private int getIndex(long value) {
-      if (value < this.binsMin) {
-        return 0;
-      } else if (value > this.bins10XMax) {
-        return this.counts.length - 1;
-      } else if (value >= this.binsMax) {
-        return this.counts.length - 2;
-      }
-      // compute the position
-      return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
-          (this.binsMax - this.binsMin));
-
-    }
-
-    /**
-     * Adds a value to the histogram.
-     */
-    public void add(long value, long count) {
-      if (value < 0) {
-        // The whole computation is completely thrown off if there are negative numbers
-        //
-        // Normally we would throw an IllegalArgumentException however this is the metrics
-        // system and it should be completely safe at all times.
-        // So silently throw it away.
-        return;
-      }
-      AtomicUtils.updateMin(min, value);
-      AtomicUtils.updateMax(max, value);
-
-      this.count.add(count);
-      this.total.add(value * count);
-
-      int pos = getIndex(value);
-      this.counts[pos].add(count);
-
-      // hasData needs to be updated as last
-      this.hasData = true;
-    }
-
-    /**
-     * Computes the quantiles give the ratios.
-     */
-    public long[] getQuantiles(double[] quantiles) {
-      if (!hasData) {
-        // No data yet.
-        return new long[quantiles.length];
-      }
-
-      // Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
-      // This is not synchronized, but since the counter are accumulating, the result is a good
-      // estimation of a snapshot.
-      long[] counts = new long[this.counts.length];
-      long total = 0L;
-      for (int i = 0; i < this.counts.length; i++) {
-        counts[i] = this.counts[i].sum();
-        total += counts[i];
-      }
-
-      int rIndex = 0;
-      double qCount = total * quantiles[0];
-      long cum = 0L;
-
-      long[] res = new long[quantiles.length];
-      countsLoop: for (int i = 0; i < counts.length; i++) {
-        // mn and mx define a value range
-        long mn, mx;
-        if (i == 0) {
-          mn = this.min.get();
-          mx = this.binsMin;
-        } else if (i == counts.length - 1) {
-          mn = this.bins10XMax;
-          mx = this.max.get();
-        } else if (i == counts.length - 2) {
-          mn = this.binsMax;
-          mx = this.bins10XMax;
-        } else {
-          mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
-          mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
-        }
-
-        if (mx < this.min.get()) {
-          continue;
-        }
-        if (mn > this.max.get()) {
-          break;
-        }
-        mn = Math.max(mn, this.min.get());
-        mx = Math.min(mx, this.max.get());
-
-        // lastCum/cum are the corresponding counts to mn/mx
-        double lastCum = cum;
-        cum += counts[i];
-
-        // fill the results for qCount is within current range.
-        while (qCount <= cum) {
-          if (cum == lastCum) {
-            res[rIndex] = mn;
-          } else {
-            res[rIndex] = (long) ((qCount - lastCum) * (mx - mn) / (cum - lastCum) + mn);
-          }
-
-          // move to next quantile
-          rIndex++;
-          if (rIndex >= quantiles.length) {
-            break countsLoop;
-          }
-          qCount = total * quantiles[rIndex];
-        }
-      }
-      // In case quantiles contains values >= 100%
-      for (; rIndex < quantiles.length; rIndex++) {
-        res[rIndex] = this.max.get();
-      }
-
-      return res;
-    }
-
-    long getNumAtOrBelow(long val) {
-      return Arrays.stream(counts).mapToLong(c -> c.sum()).limit(getIndex(val) + 1).sum();
-    }
-  }
-
-  // The bins counting values. It is replaced with a new one in calling of reset().
-  private volatile Bins bins;
-
-  /**
-   * Constructor.
-   */
-  public FastLongHistogram() {
-    this(DEFAULT_NBINS);
-  }
-
-  /**
-   * Constructor.
-   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
-   *          results but with lower efficiency, and vice versus.
-   */
-  public FastLongHistogram(int numOfBins) {
-    this.bins = new Bins(numOfBins);
-  }
-
-  /**
-   * Constructor setting the bins assuming a uniform distribution within a range.
-   * @param numOfBins the number of bins for the histogram. A larger value results in more precise
-   *          results but with lower efficiency, and vice versus.
-   * @param min lower bound of the region, inclusive.
-   * @param max higher bound of the region, inclusive.
-   */
-  public FastLongHistogram(int numOfBins, long min, long max) {
-    this(numOfBins);
-    Bins bins = new Bins(numOfBins);
-    bins.add(min, 1);
-    bins.add(max, 1);
-    this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
-  }
-
-  private FastLongHistogram(Bins bins) {
-    this.bins = bins;
-  }
-
-  /**
-   * Adds a value to the histogram.
-   */
-  public void add(long value, long count) {
-    this.bins.add(value, count);
-  }
-
-  /**
-   * Computes the quantiles give the ratios.
-   */
-  public long[] getQuantiles(double[] quantiles) {
-    return this.bins.getQuantiles(quantiles);
-  }
-
-  public long[] getQuantiles() {
-    return this.bins.getQuantiles(DEFAULT_QUANTILES);
-  }
-
-  public long getMin() {
-    long min = this.bins.min.get();
-    return min == Long.MAX_VALUE ? 0 : min; // in case it is not initialized
-  }
-
-  public long getMax() {
-    return this.bins.max.get();
-  }
-
-  public long getCount() {
-    return this.bins.count.sum();
-  }
-
-  public long getMean() {
-    Bins bins = this.bins;
-    long count = bins.count.sum();
-    long total = bins.total.sum();
-    if (count == 0) {
-      return 0;
-    }
-    return total / count;
-  }
-
-  public long getNumAtOrBelow(long value) {
-    return this.bins.getNumAtOrBelow(value);
-  }
-
-  /**
-   * Resets the histogram for new counting.
-   */
-  public FastLongHistogram reset() {
-    Bins oldBins = this.bins;
-    this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
-    return new FastLongHistogram(oldBins);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
deleted file mode 100644
index d56d143..0000000
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestFastLongHistogram.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import java.util.Arrays;
-import java.util.Random;
-
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Testcases for FastLongHistogram.
- */
-@Category({MiscTests.class, SmallTests.class})
-public class TestFastLongHistogram {
-
-  private static void doTestUniform(FastLongHistogram hist) {
-    long[] VALUES = { 0, 10, 20, 30, 40, 50 };
-    double[] qs = new double[VALUES.length];
-    for (int i = 0; i < qs.length; i++) {
-      qs[i] = (double) VALUES[i] / VALUES[VALUES.length - 1];
-    }
-
-    for (int i = 0; i < 10; i++) {
-      for (long v : VALUES) {
-        hist.add(v, 1);
-      }
-      long[] vals = hist.getQuantiles(qs);
-      System.out.println(Arrays.toString(vals));
-      for (int j = 0; j < qs.length; j++) {
-        Assert.assertTrue(j + "-th element org: " + VALUES[j] + ", act: " + vals[j],
-          Math.abs(vals[j] - VALUES[j]) <= 10);
-      }
-      hist.reset();
-    }
-  }
-
-  @Test
-  public void testUniform() {
-    FastLongHistogram hist = new FastLongHistogram(100, 0, 50);
-    doTestUniform(hist);
-  }
-
-  @Test
-  public void testAdaptionOfChange() {
-    // assumes the uniform distribution
-    FastLongHistogram hist = new FastLongHistogram(100, 0, 100);
-
-    Random rand = new Random();
-
-    for (int n = 0; n < 10; n++) {
-      for (int i = 0; i < 900; i++) {
-        hist.add(rand.nextInt(100), 1);
-      }
-
-      // add 10% outliers, this breaks the assumption, hope bin10xMax works
-      for (int i = 0; i < 100; i++) {
-        hist.add(1000 + rand.nextInt(100), 1);
-      }
-
-      long[] vals = hist.getQuantiles(new double[] { 0.25, 0.75, 0.95 });
-      System.out.println(Arrays.toString(vals));
-      if (n == 0) {
-        Assert.assertTrue("Out of possible value", vals[0] >= 0 && vals[0] <= 50);
-        Assert.assertTrue("Out of possible value", vals[1] >= 50 && vals[1] <= 100);
-        Assert.assertTrue("Out of possible value", vals[2] >= 900 && vals[2] <= 1100);
-      }
-
-      hist.reset();
-    }
-  }
-
-
-  @Test
-  public void testGetNumAtOrBelow() {
-    long[] VALUES = { 1, 10, 20, 30, 40, 50 };
-
-    FastLongHistogram h = new FastLongHistogram();
-    for (long v : VALUES) {
-      for (int i = 0; i < 100; i++) {
-        h.add(v, 1);
-      }
-    }
-
-    h.add(Integer.MAX_VALUE, 1);
-
-    h.reset();
-
-    for (long v : VALUES) {
-      for (int i = 0; i < 100; i++) {
-        h.add(v, 1);
-      }
-    }
-    // Add something way out there to make sure it doesn't throw off the counts.
-    h.add(Integer.MAX_VALUE, 1);
-
-    assertEquals(100, h.getNumAtOrBelow(1));
-    assertEquals(200, h.getNumAtOrBelow(11));
-    assertEquals(601, h.getNumAtOrBelow(Long.MAX_VALUE));
-  }
-
-
-  @Test
-  public void testSameValues() {
-    FastLongHistogram hist = new FastLongHistogram(100);
-
-    hist.add(50, 100);
-
-    hist.reset();
-    doTestUniform(hist);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
new file mode 100644
index 0000000..a5bae0a
--- /dev/null
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleMasterObserverWithMetrics.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.coprocessor.example;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.BaseMasterObserver;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+/**
+ * An example coprocessor that collects some metrics to demonstrate the usage of exporting custom
+ * metrics from the coprocessor.
+ *
+ * <p>
+ * These metrics will be available through the regular Hadoop metrics2 sinks (ganglia, opentsdb,
+ * etc) as well as JMX output. You can view a snapshot of the metrics by going to the http web UI
+ * of the master page, something like http://mymasterhost:16010/jmx
+ * </p>
+ * @see ExampleRegionObserverWithMetrics
+ */
+public class ExampleMasterObserverWithMetrics extends BaseMasterObserver {
+
+  private static final Log LOG = LogFactory.getLog(ExampleMasterObserverWithMetrics.class);
+
+  /** This is the Timer metric object to keep track of the current count across invocations */
+  private Timer createTableTimer;
+  private long createTableStartTime = Long.MIN_VALUE;
+
+  /** This is a Counter object to keep track of disableTable operations */
+  private Counter disableTableCounter;
+
+  /** Returns the total memory of the process. We will use this to define a gauge metric */
+  private long getTotalMemory() {
+    return Runtime.getRuntime().totalMemory();
+  }
+
+  /** Returns the max memory of the process. We will use this to define a gauge metric */
+  private long getMaxMemory() {
+    return Runtime.getRuntime().maxMemory();
+  }
+
+  @Override
+  public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                             HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+    super.preCreateTable(ctx, desc, regions);
+    // we rely on the fact that there is only 1 instance of our MasterObserver. We keep track of
+    // when the operation starts before the operation is executing.
+    this.createTableStartTime = System.currentTimeMillis();
+  }
+
+  @Override
+  public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+                              HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
+    super.postCreateTable(ctx, desc, regions);
+    if (this.createTableStartTime > 0) {
+      long time = System.currentTimeMillis() - this.createTableStartTime;
+      LOG.info("Create table took: " + time);
+
+      // Update the timer metric for the create table operation duration.
+      createTableTimer.updateMillis(time);
+    }
+  }
+
+  @Override
+  public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException {
+    super.preDisableTable(ctx, tableName);
+
+    // Increment the Counter for disable table operations
+    this.disableTableCounter.increment();
+  }
+
+  @Override
+  public void start(CoprocessorEnvironment env) throws IOException {
+    super.start(env);
+
+    // start for the MasterObserver will be called only once in the lifetime of the
+    // server. We will construct and register all metrics that we will track across method
+    // invocations.
+
+    if (env instanceof MasterCoprocessorEnvironment) {
+      // Obtain the MetricRegistry for the Master. Metrics from this registry will be reported
+      // at the master level per-server.
+      MetricRegistry registry =
+          ((MasterCoprocessorEnvironment) env).getMetricRegistryForMaster();
+
+      if (createTableTimer == null) {
+        // Create a new Counter, or get the already registered counter.
+        // It is much better to only call this once and save the Counter as a class field instead
+        // of creating the counter every time a coprocessor method is invoked. This will negate
+        // any performance bottleneck coming from map lookups tracking metrics in the registry.
+        createTableTimer = registry.timer("CreateTable");
+
+        // on stop(), we can remove these registered metrics via calling registry.remove(). But
+        // it is not needed for coprocessors at the master level. If coprocessor is stopped,
+        // the server is stopping anyway, so there will not be any resource leaks.
+      }
+
+      if (disableTableCounter == null) {
+        disableTableCounter = registry.counter("DisableTable");
+      }
+
+      // Register a custom gauge. The Gauge object will be registered in the metrics registry and
+      // periodically the getValue() is invoked to obtain the snapshot.
+      registry.register("totalMemory", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return getTotalMemory();
+        }
+      });
+
+      // Register a custom gauge using Java-8 lambdas (Supplier converted into Gauge)
+      registry.register("maxMemory", this::getMaxMemory);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java
new file mode 100644
index 0000000..7606b05
--- /dev/null
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ExampleRegionObserverWithMetrics.java
@@ -0,0 +1,121 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.hbase.coprocessor.example;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.Timer;
+
+/**
+ * An example coprocessor that collects some metrics to demonstrate the usage of exporting custom
+ * metrics from the coprocessor.
+ * <p>
+ * These metrics will be available through the regular Hadoop metrics2 sinks (ganglia, opentsdb,
+ * etc) as well as JMX output. You can view a snapshot of the metrics by going to the http web UI
+ * of the regionserver page, something like http://myregionserverhost:16030/jmx
+ * </p>
+ *
+ * @see ExampleMasterObserverWithMetrics
+ */
+public class ExampleRegionObserverWithMetrics extends BaseRegionObserver {
+
+  private Counter preGetCounter;
+  private Timer costlyOperationTimer;
+
+  @Override
+  public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
+      throws IOException {
+    super.preGetOp(e, get, results);
+
+    // Increment the Counter whenever the coprocessor is called
+    preGetCounter.increment();
+  }
+
+  @Override
+  public void postGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get,
+                        List<Cell> results) throws IOException {
+    super.postGetOp(e, get, results);
+
+    // do a costly (high latency) operation which we want to measure how long it takes by
+    // using a Timer (which is a Meter and a Histogram).
+    long start = System.nanoTime();
+    try {
+      performCostlyOperation();
+    } finally {
+      costlyOperationTimer.updateNanos(System.nanoTime() - start);
+    }
+  }
+
+  private void performCostlyOperation() {
+    try {
+      // simulate the operation by sleeping.
+      Thread.sleep(ThreadLocalRandom.current().nextLong(100));
+    } catch (InterruptedException ignore) {}
+  }
+
+  @Override
+  public void start(CoprocessorEnvironment env) throws IOException {
+    super.start(env);
+
+    // start for the RegionServerObserver will be called only once in the lifetime of the
+    // server. We will construct and register all metrics that we will track across method
+    // invocations.
+
+    if (env instanceof RegionCoprocessorEnvironment) {
+      // Obtain the MetricRegistry for the RegionServer. Metrics from this registry will be reported
+      // at the region server level per-regionserver.
+      MetricRegistry registry =
+          ((RegionCoprocessorEnvironment) env).getMetricRegistryForRegionServer();
+
+      if (preGetCounter == null) {
+        // Create a new Counter, or get the already registered counter.
+        // It is much better to only call this once and save the Counter as a class field instead
+        // of creating the counter every time a coprocessor method is invoked. This will negate
+        // any performance bottleneck coming from map lookups tracking metrics in the registry.
+        // Returned counter instance is shared by all coprocessors of the same class in the same
+        // region server.
+        preGetCounter = registry.counter("preGetRequests");
+      }
+
+      if (costlyOperationTimer == null) {
+        // Create a Timer to track execution times for the costly operation.
+        costlyOperationTimer = registry.timer("costlyOperation");
+      }
+    }
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    // we should NOT remove / deregister the metrics in stop(). The whole registry will be
+    // removed when the last region of the table is closed.
+    super.stop(e);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/pom.xml b/hbase-hadoop-compat/pom.xml
index 016dd24..1ec1047 100644
--- a/hbase-hadoop-compat/pom.xml
+++ b/hbase-hadoop-compat/pom.xml
@@ -122,6 +122,10 @@
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-math</artifactId>
       </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-metrics-api</artifactId>
+        </dependency>
     </dependencies>
 
     <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
index f79aa9f..652aae1 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSource.java
@@ -104,4 +104,9 @@ public interface BaseSource {
    */
   String getMetricsName();
 
+  default MetricRegistryInfo getMetricRegistryInfo() {
+    return new MetricRegistryInfo(getMetricsName(), getMetricsDescription(),
+        getMetricsContext(), getMetricsJmxContext(), true);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml
index 56c3e7f..3e63391 100644
--- a/hbase-hadoop2-compat/pom.xml
+++ b/hbase-hadoop2-compat/pom.xml
@@ -166,6 +166,10 @@ limitations under the License.
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-metrics</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
index f843ec2..6a9fff4 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/BaseSourceImpl.java
@@ -19,6 +19,9 @@
 package org.apache.hadoop.hbase.metrics;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.impl.GlobalMetricRegistriesAdapter;
+import org.apache.hadoop.hbase.metrics.impl.HBaseMetrics2HadoopMetricsAdapter;
+import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceImpl;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
@@ -47,15 +50,53 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
       inited = true;
       DefaultMetricsSystem.initialize(HBASE_METRICS_SYSTEM_NAME);
       JvmMetrics.initSingleton(name, "");
+      // initialize hbase-metrics module based metric system as well. GlobalMetricRegistriesSource
+      // initialization depends on the metric system being already initialized, that is why we are
+      // doing it here. Once BaseSourceSourceImpl is removed, we should do the initialization of
+      // these elsewhere.
+      GlobalMetricRegistriesAdapter.init();
     }
   }
 
+  /**
+   * @deprecated Use hbase-metrics/hbase-metrics-api module interfaces for new metrics.
+   * Defining BaseSources for new metric groups (WAL, RPC, etc) is not needed anymore, however,
+   * for existing BaseSource implemetnations, please use the field named "registry" which is a
+   * MetricRegistry instance together with the HBaseMetrics2HadoopMetricsAdapter.
+   */
+  @Deprecated
   protected final DynamicMetricsRegistry metricsRegistry;
   protected final String metricsName;
   protected final String metricsDescription;
   protected final String metricsContext;
   protected final String metricsJmxContext;
 
+  /**
+   * Note that there are at least 4 MetricRegistry definitions in the source code. The first one is
+   * Hadoop Metrics2 MetricRegistry, second one is DynamicMetricsRegistry which is HBase's fork
+   * of the Hadoop metrics2 class. The third one is the dropwizard metrics implementation of
+   * MetricRegistry, and finally a new API abstraction in HBase that is the
+   * o.a.h.h.metrics.MetricRegistry class. This last one is the new way to use metrics within the
+   * HBase code. However, the others are in play because of existing metrics2 based code still
+   * needs to coexists until we get rid of all of our BaseSource and convert them to the new
+   * framework. Until that happens, new metrics can use the new API, but will be collected
+   * through the HBaseMetrics2HadoopMetricsAdapter class.
+   *
+   * BaseSourceImpl has two MetricRegistries. metricRegistry is for hadoop Metrics2 based
+   * metrics, while the registry is for hbase-metrics based metrics.
+   */
+  protected final MetricRegistry registry;
+
+  /**
+   * The adapter from hbase-metrics module to metrics2. This adepter is the connection between the
+   * Metrics in the MetricRegistry and the Hadoop Metrics2 system. Using this adapter, existing
+   * BaseSource implementations can define new metrics using the hbase-metrics/hbase-metrics-api
+   * module interfaces and still be able to make use of metrics2 sinks (including JMX). Existing
+   * BaseSources should call metricsAdapter.snapshotAllMetrics() in getMetrics() method. See
+   * {@link MetricsRegionServerSourceImpl}.
+   */
+  protected final HBaseMetrics2HadoopMetricsAdapter metricsAdapter;
+
   public BaseSourceImpl(
       String metricsName,
       String metricsDescription,
@@ -72,6 +113,11 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
 
     //Register this instance.
     DefaultMetricsSystem.instance().register(metricsJmxContext, metricsDescription, this);
+
+    // hbase-metrics module based metrics are registered in the hbase MetricsRegistry.
+    registry = MetricRegistries.global().create(this.getMetricRegistryInfo());
+    metricsAdapter = new HBaseMetrics2HadoopMetricsAdapter();
+
     init();
 
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
new file mode 100644
index 0000000..ddcf56d
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/GlobalMetricRegistriesAdapter.java
@@ -0,0 +1,172 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.Optional;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystemHelper;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class acts as an adapter to export the MetricRegistry's in the global registry. Each
+ * MetricRegistry will be registered or unregistered from the metric2 system. The collection will
+ * be performed via the MetricsSourceAdapter and the MetricRegistry will collected like a
+ * BaseSource instance for a group of metrics  (like WAL, RPC, etc) with the MetricRegistryInfo's
+ * JMX context.
+ *
+ * <p>Developer note:
+ * Unlike the current metrics2 based approach, the new metrics approach
+ * (hbase-metrics-api and hbase-metrics modules) work by having different MetricRegistries that are
+ * initialized and used from the code that lives in their respective modules (hbase-server, etc).
+ * There is no need to define BaseSource classes and do a lot of indirection. The MetricRegistry'es
+ * will be in the global MetricRegistriesImpl, and this class will iterate over
+ * MetricRegistries.global() and register adapters to the metrics2 subsystem. These adapters then
+ * report the actual values by delegating to
+ * {@link HBaseMetrics2HadoopMetricsAdapter#snapshotAllMetrics(MetricRegistry, MetricsCollector)}.
+ *
+ * We do not initialize the Hadoop Metrics2 system assuming that other BaseSources already do so
+ * (see BaseSourceImpl). Once the last BaseSource is moved to the new system, the metric2
+ * initialization should be moved here.
+ * </p>
+ */
+public class GlobalMetricRegistriesAdapter {
+
+  private static final Log LOG = LogFactory.getLog(GlobalMetricRegistriesAdapter.class);
+
+  private class MetricsSourceAdapter implements MetricsSource {
+    private final MetricRegistry registry;
+    MetricsSourceAdapter(MetricRegistry registry) {
+      this.registry = registry;
+    }
+
+    @Override
+    public void getMetrics(MetricsCollector collector, boolean all) {
+      metricsAdapter.snapshotAllMetrics(registry, collector);
+    }
+  }
+
+  private final MetricsExecutor executor;
+  private final AtomicBoolean stopped;
+  private final DefaultMetricsSystemHelper helper;
+  private final HBaseMetrics2HadoopMetricsAdapter metricsAdapter;
+  private final HashMap<MetricRegistryInfo, MetricsSourceAdapter> registeredSources;
+
+  private GlobalMetricRegistriesAdapter() {
+    this.executor = new MetricsExecutorImpl();
+    this.stopped = new AtomicBoolean(false);
+    this.metricsAdapter = new HBaseMetrics2HadoopMetricsAdapter();
+    this.registeredSources = new HashMap<>();
+    this.helper = new DefaultMetricsSystemHelper();
+    executor.getExecutor().scheduleAtFixedRate(() -> this.doRun(), 10, 10, TimeUnit.SECONDS);
+  }
+
+  /**
+   * Make sure that this global MetricSource for hbase-metrics module based metrics are initialized.
+   * This should be called only once.
+   */
+  public static GlobalMetricRegistriesAdapter init() {
+    return new GlobalMetricRegistriesAdapter();
+  }
+
+  @VisibleForTesting
+  public void stop() {
+    stopped.set(true);
+  }
+
+  private void doRun() {
+    if (stopped.get()) {
+      executor.stop();
+      return;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("doRun called: " + registeredSources);
+    }
+
+    Collection<MetricRegistry> registries = MetricRegistries.global().getMetricRegistries();
+    for (MetricRegistry registry : registries) {
+      MetricRegistryInfo info = registry.getMetricRegistryInfo();
+
+      if (info.isExistingSource()) {
+        // If there is an already existing BaseSource for this MetricRegistry, skip it here. These
+        // types of registries are there only due to existing BaseSource implementations in the
+        // source code (like MetricsRegionServer, etc). This is to make sure that we can transition
+        // iteratively to the new hbase-metrics system. These type of MetricRegistry metrics will be
+        // exported from the BaseSource.getMetrics() call directly because there is already a
+        // MetricRecordBuilder there (see MetricsRegionServerSourceImpl).
+        continue;
+      }
+
+      if (!registeredSources.containsKey(info)) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Registering adapter for the MetricRegistry: " + info.getMetricsJmxContext());
+        }
+        // register this as a MetricSource under different JMX Context'es.
+        MetricsSourceAdapter adapter = new MetricsSourceAdapter(registry);
+        LOG.info("Registering " + info.getMetricsJmxContext() + " " + info.getMetricsDescription());
+        DefaultMetricsSystem.instance().register(info.getMetricsJmxContext(),
+            info.getMetricsDescription(), adapter);
+        registeredSources.put(info, adapter);
+        // next collection will collect the newly registered MetricSource. Doing this here leads to
+        // ConcurrentModificationException.
+      }
+    }
+
+    boolean removed = false;
+    // Remove registered sources if it is removed from the global registry
+    for (Iterator<Entry<MetricRegistryInfo, MetricsSourceAdapter>> it =
+         registeredSources.entrySet().iterator(); it.hasNext();) {
+      Entry<MetricRegistryInfo, MetricsSourceAdapter> entry = it.next();
+      MetricRegistryInfo info = entry.getKey();
+      Optional<MetricRegistry> found = MetricRegistries.global().get(info);
+      if (!found.isPresent()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Removing adapter for the MetricRegistry: " + info.getMetricsJmxContext());
+        }
+        synchronized(DefaultMetricsSystem.instance()) {
+          DefaultMetricsSystem.instance().unregisterSource(info.getMetricsJmxContext());
+          helper.removeSourceName(info.getMetricsJmxContext());
+          helper.removeObjectName(info.getMetricsJmxContext());
+          it.remove();
+          removed = true;
+        }
+      }
+    }
+    if (removed) {
+      JmxCacheBuster.clearJmxCache();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java
new file mode 100644
index 0000000..ec4a1a7
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/metrics/impl/HBaseMetrics2HadoopMetricsAdapter.java
@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/*
+ * Copyright 2016 Josh Elser
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics.impl;
+
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Gauge;
+import org.apache.hadoop.hbase.metrics.Histogram;
+import org.apache.hadoop.hbase.metrics.Meter;
+import org.apache.hadoop.hbase.metrics.Metric;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.metrics2.lib.MutableHistogram;
+
+/**
+ * This is the adapter from "HBase Metrics Framework", implemented in hbase-metrics-api and
+ * hbase-metrics modules to the Hadoop Metrics2 framework. This adapter is not a metric source,
+ * but a helper to be able to collect all of the Metric's in the MetricRegistry using the
+ * MetricsCollector and MetricsRecordBuilder.
+ *
+ * Some of the code is forked from https://github.com/joshelser/dropwizard-hadoop-metrics2.
+ */
+public class HBaseMetrics2HadoopMetricsAdapter {
+  private static final Log LOG
+      = LogFactory.getLog(HBaseMetrics2HadoopMetricsAdapter.class);
+  private static final String EMPTY_STRING = "";
+
+  public HBaseMetrics2HadoopMetricsAdapter() {
+  }
+
+  /**
+   * Iterates over the MetricRegistry and adds them to the {@code collector}.
+   *
+   * @param collector A metrics collector
+   */
+  public void snapshotAllMetrics(MetricRegistry metricRegistry,
+                                 MetricsCollector collector) {
+    MetricRegistryInfo info = metricRegistry.getMetricRegistryInfo();
+    MetricsRecordBuilder builder = collector.addRecord(Interns.info(info.getMetricsName(),
+        info.getMetricsDescription()));
+    builder.setContext(info.getMetricsContext());
+
+    snapshotAllMetrics(metricRegistry, builder);
+  }
+
+  /**
+   * Iterates over the MetricRegistry and adds them to the {@code builder}.
+   *
+   * @param builder A record builder
+   */
+  public void snapshotAllMetrics(MetricRegistry metricRegistry, MetricsRecordBuilder builder) {
+    Map<String, Metric> metrics = metricRegistry.getMetrics();
+
+    for (Map.Entry<String, Metric> e: metrics.entrySet()) {
+      // Always capitalize the name
+      String name = StringUtils.capitalize(e.getKey());
+      Metric metric = e.getValue();
+
+      if (metric instanceof Gauge) {
+        addGauge(name, (Gauge<?>) metric, builder);
+      } else if (metric instanceof Counter) {
+        addCounter(name, (Counter)metric, builder);
+      } else if (metric instanceof Histogram) {
+        addHistogram(name, (Histogram)metric, builder);
+      } else if (metric instanceof Meter) {
+        addMeter(name, (Meter)metric, builder);
+      } else if (metric instanceof Timer) {
+        addTimer(name, (Timer)metric, builder);
+      } else {
+        LOG.info("Ignoring unknown Metric class " + metric.getClass().getName());
+      }
+    }
+  }
+
+  private void addGauge(String name, Gauge<?> gauge, MetricsRecordBuilder builder) {
+    final MetricsInfo info = Interns.info(name, EMPTY_STRING);
+    final Object o = gauge.getValue();
+
+    // Figure out which gauge types metrics2 supports and call the right method
+    if (o instanceof Integer) {
+      builder.addGauge(info, (int) o);
+    } else if (o instanceof Long) {
+      builder.addGauge(info, (long) o);
+    } else if (o instanceof Float) {
+      builder.addGauge(info, (float) o);
+    } else if (o instanceof Double) {
+      builder.addGauge(info, (double) o);
+    } else {
+      LOG.warn("Ignoring Gauge (" + name + ") with unhandled type: " + o.getClass());
+    }
+  }
+
+  private void addCounter(String name, Counter counter, MetricsRecordBuilder builder) {
+    MetricsInfo info = Interns.info(name, EMPTY_STRING);
+    builder.addCounter(info, counter.getCount());
+  }
+
+  /**
+   * Add Histogram value-distribution data to a Hadoop-Metrics2 record building.
+   *
+   * @param builder A Hadoop-Metrics2 record builder.
+   * @param name A base name for this record.
+   * @param desc A description for this record.
+   * @param snapshot The distribution of measured values.
+   */
+  private void addHistogram(String name, Histogram histogram, MetricsRecordBuilder builder) {
+    MutableHistogram.snapshot(name, EMPTY_STRING, histogram, builder, true);
+  }
+
+  /**
+   * Add Dropwizard-Metrics rate information to a Hadoop-Metrics2 record builder, converting the
+   * rates to the appropriate unit.
+   *
+   * @param builder A Hadoop-Metrics2 record builder.
+   * @param name A base name for this record.
+   */
+  private void addMeter(String name, Meter meter, MetricsRecordBuilder builder) {
+    builder.addGauge(Interns.info(name + "_count", EMPTY_STRING), meter.getCount());
+    builder.addGauge(Interns.info(name + "_mean_rate", EMPTY_STRING), meter.getMeanRate());
+    builder.addGauge(Interns.info(name + "_1min_rate", EMPTY_STRING), meter.getOneMinuteRate());
+    builder.addGauge(Interns.info(name + "_5min_rate", EMPTY_STRING), meter.getFiveMinuteRate());
+    builder.addGauge(Interns.info(name + "_15min_rate", EMPTY_STRING),
+        meter.getFifteenMinuteRate());
+  }
+
+  private void addTimer(String name, Timer timer, MetricsRecordBuilder builder) {
+    addMeter(name, timer.getMeter(), builder);
+    addHistogram(name, timer.getHistogram(), builder);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
index b412fd1..67764f8 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java
@@ -510,6 +510,12 @@ public class MetricsRegionServerSourceImpl
     }
 
     metricsRegistry.snapshot(mrb, all);
+
+    // source is registered in supers constructor, sometimes called before the whole initialization.
+    if (metricsAdapter != null) {
+      // snapshot MetricRegistry as well
+      metricsAdapter.snapshotAllMetrics(registry, mrb);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
index 832e220..eb465c3 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DefaultMetricsSystemHelper.java
@@ -17,26 +17,44 @@
  */
 package org.apache.hadoop.metrics2.lib;
 
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import java.lang.reflect.Method;
-
 public class DefaultMetricsSystemHelper {
 
   private static final Log LOG = LogFactory.getLog(DefaultMetricsSystemHelper.class);
   private final Method removeObjectMethod;
+  private final Field sourceNamesField;
+  private final Field mapField;
 
   public DefaultMetricsSystemHelper() {
+    Class<? extends DefaultMetricsSystem> clazz = DefaultMetricsSystem.INSTANCE.getClass();
     Method m;
     try {
-      Class<? extends DefaultMetricsSystem> clazz = DefaultMetricsSystem.INSTANCE.getClass();
       m = clazz.getDeclaredMethod("removeObjectName", String.class);
       m.setAccessible(true);
     } catch (NoSuchMethodException e) {
       m = null;
     }
     removeObjectMethod = m;
+
+    Field f1, f2;
+    try {
+      f1 = clazz.getDeclaredField("sourceNames");
+      f1.setAccessible(true);
+      f2 = UniqueNames.class.getDeclaredField("map");
+      f2.setAccessible(true);
+    } catch (NoSuchFieldException e) {
+      LOG.trace(e);
+      f1 = null;
+      f2 = null;
+    }
+    sourceNamesField = f1;
+    mapField = f2;
   }
 
   public boolean removeObjectName(final String name) {
@@ -52,4 +70,30 @@ public class DefaultMetricsSystemHelper {
     }
     return false;
   }
+
+  /**
+   * Unfortunately Hadoop tries to be too-clever and permanently keeps track of all names registered
+   * so far as a Source, thus preventing further re-registration of the source with the same name.
+   * In case of dynamic metrics tied to region-lifecycles, this becomes a problem because we would
+   * like to be able to re-register and remove with the same name. Otherwise, it is resource leak.
+   * This ugly code manually removes the name from the UniqueNames map.
+   * TODO: May not be needed for Hadoop versions after YARN-5190.
+   */
+  public void removeSourceName(String name) {
+    if (sourceNamesField == null || mapField == null) {
+      return;
+    }
+    try {
+      Object sourceNames = sourceNamesField.get(DefaultMetricsSystem.INSTANCE);
+      HashMap map = (HashMap) mapField.get(sourceNames);
+      synchronized (sourceNames) {
+        map.remove(name);
+      }
+    } catch (Exception ex) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Received exception while trying to access Hadoop Metrics classes via reflection.",
+            ex);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
index ec8cb50..6a2f203 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableHistogram.java
@@ -18,12 +18,12 @@
 
 package org.apache.hadoop.metrics2.lib;
 
-import java.util.concurrent.atomic.LongAdder;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.metrics.Histogram;
 import org.apache.hadoop.hbase.metrics.Interns;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.hadoop.hbase.metrics.impl.HistogramImpl;
 import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -33,28 +33,9 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
  */
 @InterfaceAudience.Private
 public class MutableHistogram extends MutableMetric implements MetricHistogram {
-  // Double buffer the two FastLongHistograms.
-  // As they are reset they learn how the buckets should be spaced
-  // So keep two around and use them
-  protected final FastLongHistogram histogram;
-
+  protected HistogramImpl histogram;
   protected final String name;
   protected final String desc;
-  protected final LongAdder counter = new LongAdder();
-
-  private boolean metricsInfoStringInited = false;
-  private String NUM_OPS_METRIC;
-  private String MIN_METRIC;
-  private String MAX_METRIC;
-  private String MEAN_METRIC;
-  private String MEDIAN_METRIC;
-  private String TWENTY_FIFTH_PERCENTILE_METRIC;
-  private String SEVENTY_FIFTH_PERCENTILE_METRIC;
-  private String NINETIETH_PERCENTILE_METRIC;
-  private String NINETY_FIFTH_PERCENTILE_METRIC;
-  private String NINETY_EIGHTH_PERCENTILE_METRIC;
-  private String NINETY_NINETH_PERCENTILE_METRIC;
-  private String NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC;
 
   public MutableHistogram(MetricsInfo info) {
     this(info.name(), info.description());
@@ -67,12 +48,11 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
   protected MutableHistogram(String name, String description, long maxExpected) {
     this.name = StringUtils.capitalize(name);
     this.desc = StringUtils.uncapitalize(description);
-    this.histogram = new FastLongHistogram(FastLongHistogram.DEFAULT_NBINS, 1, maxExpected);
+    this.histogram = new HistogramImpl();
   }
 
   public void add(final long val) {
-    counter.increment();
-    histogram.add(val, 1);
+    histogram.update(val);
   }
 
   public long getMax() {
@@ -81,56 +61,42 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
 
   @Override
   public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
-    // Get a reference to the old histogram.
-    FastLongHistogram histo = histogram.reset();
-    if (histo != null) {
-      updateSnapshotMetrics(metricsRecordBuilder, histo);
-    }
+    snapshot(name, desc, histogram, metricsRecordBuilder, all);
   }
 
-  protected void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder,
-                                       FastLongHistogram histo) {
-    if (!metricsInfoStringInited) {
-      NUM_OPS_METRIC = name + NUM_OPS_METRIC_NAME;
-      MIN_METRIC = name + MIN_METRIC_NAME;
-      MAX_METRIC = name + MAX_METRIC_NAME;
-      MEAN_METRIC = name + MEAN_METRIC_NAME;
-      MEDIAN_METRIC = name + MEDIAN_METRIC_NAME;
-      TWENTY_FIFTH_PERCENTILE_METRIC = name + TWENTY_FIFTH_PERCENTILE_METRIC_NAME;
-      SEVENTY_FIFTH_PERCENTILE_METRIC = name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME;
-      NINETIETH_PERCENTILE_METRIC = name + NINETIETH_PERCENTILE_METRIC_NAME;
-      NINETY_FIFTH_PERCENTILE_METRIC = name + NINETY_FIFTH_PERCENTILE_METRIC_NAME;
-      NINETY_EIGHTH_PERCENTILE_METRIC = name + NINETY_EIGHTH_PERCENTILE_METRIC_NAME;
-      NINETY_NINETH_PERCENTILE_METRIC = name + NINETY_NINETH_PERCENTILE_METRIC_NAME;
-      NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC = name +
-          NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME;
-
-      metricsInfoStringInited = true;
+  public static void snapshot(String name, String desc, Histogram histogram,
+                              MetricsRecordBuilder metricsRecordBuilder, boolean all) {
+    // Get a reference to the old histogram.
+    Snapshot snapshot = histogram.snapshot();
+    if (snapshot != null) {
+      updateSnapshotMetrics(name, desc, histogram, snapshot, metricsRecordBuilder);
     }
+  }
 
-    metricsRecordBuilder.addCounter(Interns.info(NUM_OPS_METRIC, desc), counter.sum());
-    metricsRecordBuilder.addGauge(Interns.info(MIN_METRIC, desc), histo.getMin());
-    metricsRecordBuilder.addGauge(Interns.info(MAX_METRIC, desc), histo.getMax());
-    metricsRecordBuilder.addGauge(Interns.info(MEAN_METRIC, desc), histo.getMean());
-
-    long[] percentiles = histo.getQuantiles();
-
-    metricsRecordBuilder.addGauge(Interns.info(TWENTY_FIFTH_PERCENTILE_METRIC, desc),
-        percentiles[0]);
-    metricsRecordBuilder.addGauge(Interns.info(MEDIAN_METRIC, desc),
-        percentiles[1]);
-    metricsRecordBuilder.addGauge(Interns.info(SEVENTY_FIFTH_PERCENTILE_METRIC, desc),
-        percentiles[2]);
-    metricsRecordBuilder.addGauge(Interns.info(NINETIETH_PERCENTILE_METRIC, desc),
-        percentiles[3]);
-    metricsRecordBuilder.addGauge(Interns.info(NINETY_FIFTH_PERCENTILE_METRIC, desc),
-        percentiles[4]);
-    metricsRecordBuilder.addGauge(Interns.info(NINETY_EIGHTH_PERCENTILE_METRIC, desc),
-        percentiles[5]);
-    metricsRecordBuilder.addGauge(Interns.info(NINETY_NINETH_PERCENTILE_METRIC, desc),
-        percentiles[6]);
+  protected static void updateSnapshotMetrics(String name, String desc, Histogram histogram,
+      Snapshot snapshot, MetricsRecordBuilder metricsRecordBuilder) {
+    metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc),
+        histogram.getCount());
+    metricsRecordBuilder.addGauge(Interns.info(name + MIN_METRIC_NAME, desc), snapshot.getMin());
+    metricsRecordBuilder.addGauge(Interns.info(name + MAX_METRIC_NAME, desc), snapshot.getMax());
+    metricsRecordBuilder.addGauge(Interns.info(name + MEAN_METRIC_NAME, desc), snapshot.getMean());
+
+    metricsRecordBuilder.addGauge(Interns.info(name + TWENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get25thPercentile());
+    metricsRecordBuilder.addGauge(Interns.info(name + MEDIAN_METRIC_NAME, desc),
+        snapshot.getMedian());
+    metricsRecordBuilder.addGauge(Interns.info(name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get75thPercentile());
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETIETH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get90thPercentile());
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETY_FIFTH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get95thPercentile());
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETY_EIGHTH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get98thPercentile());
+    metricsRecordBuilder.addGauge(Interns.info(name + NINETY_NINETH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get99thPercentile());
     metricsRecordBuilder.addGauge(
-        Interns.info(NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC, desc),
-        percentiles[7]);
+        Interns.info(name + NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME, desc),
+        snapshot.get999thPercentile());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
index 94bcdaa..a12dc27 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.metrics2.lib;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.Interns;
-import org.apache.hadoop.hbase.util.FastLongHistogram;
+import org.apache.hadoop.hbase.metrics.Snapshot;
 import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.MetricsInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -56,22 +56,22 @@ public abstract class MutableRangeHistogram extends MutableHistogram implements
   @Override
   public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
     // Get a reference to the old histogram.
-    FastLongHistogram histo = histogram.reset();
-    if (histo != null) {
-      updateSnapshotMetrics(metricsRecordBuilder, histo);
-      updateSnapshotRangeMetrics(metricsRecordBuilder, histo);
+    Snapshot snapshot = histogram.snapshot();
+    if (snapshot != null) {
+      updateSnapshotMetrics(name, desc, histogram, snapshot, metricsRecordBuilder);
+      updateSnapshotRangeMetrics(metricsRecordBuilder, snapshot);
     }
   }
 
   public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder,
-                                         FastLongHistogram histogram) {
+                                         Snapshot snapshot) {
     long priorRange = 0;
     long cumNum = 0;
 
     final long[] ranges = getRanges();
     final String rangeType = getRangeType();
     for (int i = 0; i < ranges.length - 1; i++) {
-      long val = histogram.getNumAtOrBelow(ranges[i]);
+      long val = snapshot.getCountAtOrBelow(ranges[i]);
       if (val - cumNum > 0) {
         metricsRecordBuilder.addCounter(
             Interns.info(name + "_" + rangeType + "_" + priorRange + "-" + ranges[i], desc),

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/README.txt
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/README.txt b/hbase-metrics-api/README.txt
new file mode 100644
index 0000000..dfaa29f
--- /dev/null
+++ b/hbase-metrics-api/README.txt
@@ -0,0 +1,78 @@
+Overview
+========
+hbase-metrics and hbase-metrics-api are two modules that define and implement the "new" metric
+system used internally within HBase. These two modules (and some other code in hbase-hadoop2-compat)
+module are referred as "HBase metrics framework".
+
+HBase-metrics-api Module
+========================
+HBase Metrics API (hbase-metrics-api) contains the interface
+that HBase exposes internally and to third party code (including coprocessors). It is a thin
+abstraction over the actual implementation for backwards compatibility guarantees. The source
+/ binary and other compatibility guarantees are for "LimitedPrivate API" (see [1] for an
+explanation).
+
+The metrics API in this hbase-metrics-api module is inspired by the Dropwizard metrics 3.1 API
+(See [2]). It is a subset of the API only containing metrics collection. However, the implementation
+is HBase-specific and provided in hbase-metrics module. All of the classes in this module is
+HBase-internal. See the latest documentation of Dropwizard metrics for examples of defining / using
+metrics.
+
+
+HBase-metrics Module
+====================
+hbase-metrics module contains implementation of the "HBase Metrics API", including MetricRegistry,
+Counter, Histogram, etc. These are highly concurrent implementations of the Metric interfaces.
+Metrics in HBase are grouped into different sets (like WAL, RPC, RegionServer, etc). Each group of
+metrics should be tracked via a MetricRegistry specific to that group. Metrics can be dynamically
+added or removed from the registry with a name. Each Registry is independent of the other
+registries and will have it's own JMX context and MetricRecord (when used with Metrics2).
+
+
+MetricRegistry's themselves are tracked via a global registry (of MetricRegistries) called
+MetricRegistries. MetricRegistries.global() can be used to obtain the global instance.
+MetricRegistry instances can also be dynamically registered and removed. However, unlike the
+MetricRegistry, MetricRegistries does reference counting of the MetricRegistry instances. Only
+Metrics in the MetricRegistry instances that are in the global MetricRegistry are exported to the
+metric sinks or JMX.
+
+
+Coprocessor Metrics
+===================
+HBase allows custom coprocessors to track and export metrics using the new framework.
+Coprocessors and other third party code should only use the classes and interfaces from
+hbase-metrics-api module and only the classes that are marked with InterfaceAudience.LimitedPrivate
+annotation. There is no guarantee on the compatibility requirements for other classes.
+
+Coprocessors can obtain the MetricRegistry to register their custom metrics via corresponding
+CoprocessorEnvironment context. See ExampleRegionObserverWithMetrics and
+ExampleMasterObserverWithMetrics classes in hbase-examples module for usage.
+
+
+Developer Notes
+===============
+Historically, HBase has been using Hadoop's Metrics2 framework [3] for collecting and reporting the
+metrics internally. However, due to the difficultly of dealing with the Metrics2 framework, HBase is
+moving away from Hadoop's metrics implementation to its custom implementation. The move will happen
+incrementally, and during the time, both Hadoop Metrics2-based metrics and hbase-metrics module
+based classes will be in the source code. All new implementations for metrics SHOULD use the new
+API and framework.
+
+Examples of the new framework can be found in MetricsCoprocessor and MetricsRegionServerSourceImpl
+classes. See HBASE-9774 [4] for more context.
+
+hbase-metrics module right now only deals with metrics tracking and collection. It does not do JMX
+reporting or reporting to console, ganglia, opentsdb, etc. We use Hadoop's Metrics2 for reporting
+metrics to different sinks or exporting via JMX. However, this is contained within the
+hbase-hadoop2-compat module completely, so that rest of the code does not know anything about the
+Metrics2 dependency. HBaseMetrics2HadoopMetricsAdapter is the adapter that can collect metrics
+in a MetricRegistry using the metric2 MetricsCollector / MetricRecordBuilder interfaces.
+GlobalMetricRegistriesSource is the global Metrics2 Source that collects all of the metrics in all
+of the metric registries in the MetricRegistries.global() instance.
+
+
+References
+1. https://hbase.apache.org/book.html#hbase.versioning
+2. http://metrics.dropwizard.io/
+3. https://hadoop.apache.org/docs/r2.7.2/api/org/apache/hadoop/metrics2/package-summary.html
+4. https://issues.apache.org/jira/browse/HBASE-9774
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/pom.xml b/hbase-metrics-api/pom.xml
new file mode 100644
index 0000000..5822971
--- /dev/null
+++ b/hbase-metrics-api/pom.xml
@@ -0,0 +1,112 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+  /**
+   * Licensed to the Apache Software Foundation (ASF) under one
+   * or more contributor license agreements.  See the NOTICE file
+   * distributed with this work for additional information
+   * regarding copyright ownership.  The ASF licenses this file
+   * to you under the Apache License, Version 2.0 (the
+   * "License"); you may not use this file except in compliance
+   * with the License.  You may obtain a copy of the License at
+   *
+   *     http://www.apache.org/licenses/LICENSE-2.0
+   *
+   * Unless required by applicable law or agreed to in writing, software
+   * distributed under the License is distributed on an "AS IS" BASIS,
+   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   * See the License for the specific language governing permissions and
+   * limitations under the License.
+   */
+  -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+
+  <artifactId>hbase-metrics-api</artifactId>
+  <name>Apache HBase - Metrics API</name>
+  <description>HBase Metrics API descriptions</description>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven.assembly.version}</version>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <!-- Always skip the second part executions, since we only run simple unit tests in this module -->
+        <executions>
+          <execution>
+            <id>secondPartTestsExecution</id>
+            <phase>test</phase>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+    <!-- Skip the tests in this module -->
+    <profile>
+      <id>skip-metrics-api-tests</id>
+      <activation>
+        <property>
+          <name>skip-metrics-api-tests</name>
+        </property>
+      </activation>
+      <properties>
+        <surefire.skipFirstPart>true</surefire.skipFirstPart>
+      </properties>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java
new file mode 100644
index 0000000..2e4147e
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Counter.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A mutable number optimized for high concurrency counting.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Counter extends Metric {
+
+  /**
+   * Increment {@code this} by 1.
+   */
+  void increment();
+
+  /**
+   * Increment {@code this} by {@code n}.
+   *
+   * @param n The amount to increment.
+   */
+  void increment(long n);
+
+  /**
+   * Decrement {@code this} by 1.
+   */
+  void decrement();
+
+  /**
+   * Decrement {@code this} by {@code n}.
+   *
+   * @param n The amount to decrement.
+   */
+  void decrement(long n);
+
+  /**
+   * Returns the current value.
+   * @return the current value.
+   */
+  long getCount();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java
new file mode 100644
index 0000000..90df8e0
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Gauge.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metrics which measures a discrete value.
+ *
+ * @param <T> The value of the Gauge.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Gauge<T> extends Metric {
+
+  T getValue();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/c64a1d19/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java
----------------------------------------------------------------------
diff --git a/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java
new file mode 100644
index 0000000..b5b54c7
--- /dev/null
+++ b/hbase-metrics-api/src/main/java/org/apache/hadoop/hbase/metrics/Histogram.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.metrics;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * A metric which measures the distribution of values.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
+public interface Histogram extends Metric {
+
+  /**
+   * Adds a new value to the distribution.
+   *
+   * @param value The value to add
+   */
+  void update(int value);
+
+  /**
+   * Adds a new value to the distribution.
+   *
+   * @param value The value to add
+   */
+  void update(long value);
+
+  /**
+   * Return the total number of values added to the histogram.
+   * @return the total number of values.
+   */
+  long getCount();
+
+  /**
+   * Snapshot the current values in the Histogram
+   * @return a Snapshot of the distribution.
+   */
+  @InterfaceAudience.Private
+  Snapshot snapshot();
+
+}