You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2015/12/09 03:04:01 UTC

[2/4] hbase git commit: HBASE-14869 Better request latency and size histograms. (Vikas Vishwakarma and Lars Hofhansl)

HBASE-14869 Better request latency and size histograms. (Vikas Vishwakarma and Lars Hofhansl)


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

Branch: refs/heads/master
Commit: 7bfbb6a3c9af4b0e2853b5ea2580a05bb471211b
Parents: 2430efc
Author: Lars Hofhansl <la...@apache.org>
Authored: Tue Dec 8 17:02:27 2015 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Tue Dec 8 17:02:27 2015 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/test/MetricsAssertHelper.java  | 10 +++
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java | 10 +--
 .../MetricsAssignmentManagerSourceImpl.java     |  4 +-
 .../MetricsMasterFilesystemSourceImpl.java      | 10 ++-
 .../hbase/master/MetricsSnapshotSourceImpl.java |  6 +-
 .../balancer/MetricsBalancerSourceImpl.java     |  2 +-
 .../MetricsRegionServerSourceImpl.java          | 18 ++--
 .../regionserver/MetricsRegionSourceImpl.java   |  4 +-
 .../wal/MetricsEditsReplaySourceImpl.java       |  6 +-
 .../regionserver/wal/MetricsWALSourceImpl.java  |  6 +-
 .../thrift/MetricsThriftServerSourceImpl.java   | 10 +--
 .../metrics2/lib/DynamicMetricsRegistry.java    | 40 +++++++++
 .../hadoop/metrics2/lib/MutableHistogram.java   | 10 ++-
 .../metrics2/lib/MutableRangeHistogram.java     | 94 ++++++++++++++++++++
 .../metrics2/lib/MutableSizeHistogram.java      | 57 ++++++++++++
 .../metrics2/lib/MutableTimeHistogram.java      | 58 ++++++++++++
 .../hbase/test/MetricsAssertHelperImpl.java     |  8 +-
 .../regionserver/TestRegionServerMetrics.java   | 67 ++++++++++++++
 18 files changed, 379 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java
index 2eefcd2..70f77f1 100644
--- a/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java
+++ b/hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelper.java
@@ -140,6 +140,16 @@ public interface MetricsAssertHelper {
   long getCounter(String name, BaseSource source);
 
   /**
+   * Check if a dynamic counter exists.
+   *
+   * @param name   name of the counter.
+   * @param source The BaseSource{@link BaseSource} that will provide the tags,
+   *               gauges, and counters.
+   * @return boolean true id counter metric exists.
+   */
+  boolean checkCounterExists(String name, BaseSource source);
+
+  /**
    * Get the value of a gauge as a double.
    *
    * @param name   name of the gauge.

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
index 78b1c66..8984394 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
@@ -92,15 +92,15 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
         SENT_BYTES_DESC, 0L);
     this.receivedBytes = this.getMetricsRegistry().newCounter(RECEIVED_BYTES_NAME,
         RECEIVED_BYTES_DESC, 0L);
-    this.queueCallTime = this.getMetricsRegistry().newHistogram(QUEUE_CALL_TIME_NAME,
+    this.queueCallTime = this.getMetricsRegistry().newTimeHistogram(QUEUE_CALL_TIME_NAME,
         QUEUE_CALL_TIME_DESC);
-    this.processCallTime = this.getMetricsRegistry().newHistogram(PROCESS_CALL_TIME_NAME,
+    this.processCallTime = this.getMetricsRegistry().newTimeHistogram(PROCESS_CALL_TIME_NAME,
         PROCESS_CALL_TIME_DESC);
-    this.totalCallTime = this.getMetricsRegistry().newHistogram(TOTAL_CALL_TIME_NAME,
+    this.totalCallTime = this.getMetricsRegistry().newTimeHistogram(TOTAL_CALL_TIME_NAME,
         TOTAL_CALL_TIME_DESC);
-    this.requestSize = this.getMetricsRegistry().newHistogram(REQUEST_SIZE_NAME,
+    this.requestSize = this.getMetricsRegistry().newSizeHistogram(REQUEST_SIZE_NAME,
         REQUEST_SIZE_DESC);
-    this.responseSize = this.getMetricsRegistry().newHistogram(RESPONSE_SIZE_NAME,
+    this.responseSize = this.getMetricsRegistry().newSizeHistogram(RESPONSE_SIZE_NAME,
               RESPONSE_SIZE_DESC);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
index 215855f..ccf1c1d 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsAssignmentManagerSourceImpl.java
@@ -46,8 +46,8 @@ public class MetricsAssignmentManagerSourceImpl extends BaseSourceImpl implement
     ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, "", 0l);
     ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME, "", 0l);
     ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, "", 0l);
-    assignTimeHisto = metricsRegistry.newHistogram(ASSIGN_TIME_NAME);
-    bulkAssignTimeHisto = metricsRegistry.newHistogram(BULK_ASSIGN_TIME_NAME);
+    assignTimeHisto = metricsRegistry.newTimeHistogram(ASSIGN_TIME_NAME);
+    bulkAssignTimeHisto = metricsRegistry.newTimeHistogram(BULK_ASSIGN_TIME_NAME);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
index 1644207..28414ea 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFilesystemSourceImpl.java
@@ -42,10 +42,12 @@ public class MetricsMasterFilesystemSourceImpl extends BaseSourceImpl implements
 
   @Override
   public void init() {
-    splitSizeHisto = metricsRegistry.newHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
-    splitTimeHisto = metricsRegistry.newHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
-    metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
-    metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
+    splitSizeHisto = metricsRegistry.newSizeHistogram(SPLIT_SIZE_NAME, SPLIT_SIZE_DESC);
+    splitTimeHisto = metricsRegistry.newTimeHistogram(SPLIT_TIME_NAME, SPLIT_TIME_DESC);
+    metaSplitTimeHisto =
+        metricsRegistry.newTimeHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC);
+    metaSplitSizeHisto =
+        metricsRegistry.newSizeHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
index 80e1bad..c2fc6b9 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsSnapshotSourceImpl.java
@@ -41,11 +41,11 @@ public class MetricsSnapshotSourceImpl extends BaseSourceImpl implements Metrics
 
   @Override
   public void init() {
-    snapshotTimeHisto = metricsRegistry.newHistogram(
+    snapshotTimeHisto = metricsRegistry.newTimeHistogram(
         SNAPSHOT_TIME_NAME, SNAPSHOT_TIME_DESC);
-    snapshotCloneTimeHisto = metricsRegistry.newHistogram(
+    snapshotCloneTimeHisto = metricsRegistry.newTimeHistogram(
         SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC);
-    snapshotRestoreTimeHisto = metricsRegistry.newHistogram(
+    snapshotRestoreTimeHisto = metricsRegistry.newTimeHistogram(
         SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
index 774e2e7..da34df2 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/balancer/MetricsBalancerSourceImpl.java
@@ -42,7 +42,7 @@ public class MetricsBalancerSourceImpl extends BaseSourceImpl implements Metrics
 
   @Override
   public void init() {
-    blanceClusterHisto = metricsRegistry.newHistogram(BALANCE_CLUSTER);
+    blanceClusterHisto = metricsRegistry.newTimeHistogram(BALANCE_CLUSTER);
     miscCount = metricsRegistry.newCounter(MISC_INVOATION_COUNT, "", 0L);
 
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/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 00a3b8b..f40811c 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
@@ -68,26 +68,26 @@ public class MetricsRegionServerSourceImpl
     super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
     this.rsWrap = rsWrap;
 
-    putHisto = getMetricsRegistry().newHistogram(MUTATE_KEY);
+    putHisto = getMetricsRegistry().newTimeHistogram(MUTATE_KEY);
     slowPut = getMetricsRegistry().newCounter(SLOW_MUTATE_KEY, SLOW_MUTATE_DESC, 0L);
 
-    deleteHisto = getMetricsRegistry().newHistogram(DELETE_KEY);
+    deleteHisto = getMetricsRegistry().newTimeHistogram(DELETE_KEY);
     slowDelete = getMetricsRegistry().newCounter(SLOW_DELETE_KEY, SLOW_DELETE_DESC, 0L);
 
-    getHisto = getMetricsRegistry().newHistogram(GET_KEY);
+    getHisto = getMetricsRegistry().newTimeHistogram(GET_KEY);
     slowGet = getMetricsRegistry().newCounter(SLOW_GET_KEY, SLOW_GET_DESC, 0L);
 
-    incrementHisto = getMetricsRegistry().newHistogram(INCREMENT_KEY);
+    incrementHisto = getMetricsRegistry().newTimeHistogram(INCREMENT_KEY);
     slowIncrement = getMetricsRegistry().newCounter(SLOW_INCREMENT_KEY, SLOW_INCREMENT_DESC, 0L);
 
-    appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY);
+    appendHisto = getMetricsRegistry().newTimeHistogram(APPEND_KEY);
     slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0L);
     
-    replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY);
-    scanNextHisto = getMetricsRegistry().newHistogram(SCAN_NEXT_KEY);
+    replayHisto = getMetricsRegistry().newTimeHistogram(REPLAY_KEY);
+    scanNextHisto = getMetricsRegistry().newTimeHistogram(SCAN_NEXT_KEY);
 
-    splitTimeHisto = getMetricsRegistry().newHistogram(SPLIT_KEY);
-    flushTimeHisto = getMetricsRegistry().newHistogram(FLUSH_KEY);
+    splitTimeHisto = getMetricsRegistry().newTimeHistogram(SPLIT_KEY);
+    flushTimeHisto = getMetricsRegistry().newTimeHistogram(FLUSH_KEY);
 
     splitRequest = getMetricsRegistry().newCounter(SPLIT_REQUEST_KEY, SPLIT_REQUEST_DESC, 0L);
     splitSuccess = getMetricsRegistry().newCounter(SPLIT_SUCCESS_KEY, SPLIT_SUCCESS_DESC, 0L);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
index b31e71d..1df72d5 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
@@ -92,10 +92,10 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
     regionAppend = registry.getLongCounter(regionAppendKey, 0L);
 
     regionGetKey = regionNamePrefix + MetricsRegionServerSource.GET_KEY;
-    regionGet = registry.newHistogram(regionGetKey);
+    regionGet = registry.newTimeHistogram(regionGetKey);
 
     regionScanNextKey = regionNamePrefix + MetricsRegionServerSource.SCAN_NEXT_KEY;
-    regionScanNext = registry.newHistogram(regionScanNextKey);
+    regionScanNext = registry.newTimeHistogram(regionScanNextKey);
 
     hashCode = regionWrapper.getRegionHashCode();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java
index b10a69e..787fe76 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java
@@ -52,11 +52,11 @@ public class MetricsEditsReplaySourceImpl extends BaseSourceImpl implements
   @Override
   public void init() {
     super.init();
-    replayTimeHisto = metricsRegistry.newHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC);
-    replayBatchSizeHisto = metricsRegistry.newHistogram(REPLAY_BATCH_SIZE_NAME,
+    replayTimeHisto = metricsRegistry.newTimeHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC);
+    replayBatchSizeHisto = metricsRegistry.newSizeHistogram(REPLAY_BATCH_SIZE_NAME,
       REPLAY_BATCH_SIZE_DESC);
     replayDataSizeHisto = metricsRegistry
-        .newHistogram(REPLAY_DATA_SIZE_NAME, REPLAY_DATA_SIZE_DESC);
+        .newSizeHistogram(REPLAY_DATA_SIZE_NAME, REPLAY_DATA_SIZE_DESC);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
index 7583780..a149d1b 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALSourceImpl.java
@@ -52,12 +52,12 @@ public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSo
     super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
 
     //Create and store the metrics that will be used.
-    appendTimeHisto = this.getMetricsRegistry().newHistogram(APPEND_TIME, APPEND_TIME_DESC);
-    appendSizeHisto = this.getMetricsRegistry().newHistogram(APPEND_SIZE, APPEND_SIZE_DESC);
+    appendTimeHisto = this.getMetricsRegistry().newTimeHistogram(APPEND_TIME, APPEND_TIME_DESC);
+    appendSizeHisto = this.getMetricsRegistry().newSizeHistogram(APPEND_SIZE, APPEND_SIZE_DESC);
     appendCount = this.getMetricsRegistry().newCounter(APPEND_COUNT, APPEND_COUNT_DESC, 0l);
     slowAppendCount =
         this.getMetricsRegistry().newCounter(SLOW_APPEND_COUNT, SLOW_APPEND_COUNT_DESC, 0l);
-    syncTimeHisto = this.getMetricsRegistry().newHistogram(SYNC_TIME, SYNC_TIME_DESC);
+    syncTimeHisto = this.getMetricsRegistry().newTimeHistogram(SYNC_TIME, SYNC_TIME_DESC);
     logRollRequested =
         this.getMetricsRegistry().newCounter(ROLL_REQUESTED, ROLL_REQUESTED_DESC, 0L);
     lowReplicationLogRollRequested = this.getMetricsRegistry()

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
index e572e19..f9612e5 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/thrift/MetricsThriftServerSourceImpl.java
@@ -51,11 +51,11 @@ public class MetricsThriftServerSourceImpl extends BaseSourceImpl implements
   @Override
   public void init() {
     super.init();
-    batchGetStat = getMetricsRegistry().newHistogram(BATCH_GET_KEY);
-    batchMutateStat = getMetricsRegistry().newHistogram(BATCH_MUTATE_KEY);
-    queueTimeStat = getMetricsRegistry().newHistogram(TIME_IN_QUEUE_KEY);
-    thriftCallStat = getMetricsRegistry().newHistogram(THRIFT_CALL_KEY);
-    thriftSlowCallStat = getMetricsRegistry().newHistogram(SLOW_THRIFT_CALL_KEY);
+    batchGetStat = getMetricsRegistry().newTimeHistogram(BATCH_GET_KEY);
+    batchMutateStat = getMetricsRegistry().newTimeHistogram(BATCH_MUTATE_KEY);
+    queueTimeStat = getMetricsRegistry().newTimeHistogram(TIME_IN_QUEUE_KEY);
+    thriftCallStat = getMetricsRegistry().newTimeHistogram(THRIFT_CALL_KEY);
+    thriftSlowCallStat = getMetricsRegistry().newTimeHistogram(SLOW_THRIFT_CALL_KEY);
     callQueueLenGauge = getMetricsRegistry().getLongGauge(CALL_QUEUE_LEN_KEY, 0);
 
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
index d491236..ee13c76 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
@@ -285,6 +285,46 @@ public class DynamicMetricsRegistry {
     MutableHistogram histo = new MutableHistogram(name, desc);
     return addNewMetricIfAbsent(name, histo, MutableHistogram.class);
   }
+  
+  /**
+   * Create a new histogram with time range counts.
+   * @param name Name of the histogram.
+   * @return A new MutableTimeHistogram
+   */
+  public MutableTimeHistogram newTimeHistogram(String name) {
+     return newTimeHistogram(name, "");
+  }
+
+  /**
+   * Create a new histogram with time range counts.
+   * @param name The name of the histogram
+   * @param desc The description of the data in the histogram.
+   * @return A new MutableTimeHistogram
+   */
+  public MutableTimeHistogram newTimeHistogram(String name, String desc) {
+    MutableTimeHistogram histo = new MutableTimeHistogram(name, desc);
+    return addNewMetricIfAbsent(name, histo, MutableTimeHistogram.class);
+  }
+  
+  /**
+   * Create a new histogram with size range counts.
+   * @param name Name of the histogram.
+   * @return A new MutableSizeHistogram
+   */
+  public MutableSizeHistogram newSizeHistogram(String name) {
+     return newSizeHistogram(name, "");
+  }
+
+  /**
+   * Create a new histogram with size range counts.
+   * @param name The name of the histogram
+   * @param desc The description of the data in the histogram.
+   * @return A new MutableSizeHistogram
+   */
+  public MutableSizeHistogram newSizeHistogram(String name, String desc) {
+    MutableSizeHistogram histo = new MutableSizeHistogram(name, desc);
+    return addNewMetricIfAbsent(name, histo, MutableSizeHistogram.class);
+  }
 
   /**
    * Create a new MutableQuantile(A more accurate histogram).

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/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 6d85542..de29940 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
@@ -41,8 +41,8 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
   // Per Cormode et al. an alpha of 0.015 strongly biases to the last 5 minutes
   private static final double DEFAULT_ALPHA = 0.015;
 
-  private final String name;
-  private final String desc;
+  protected final String name;
+  protected final String desc;
   private final Sample sample;
   private final AtomicLong min;
   private final AtomicLong max;
@@ -116,6 +116,11 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
   public void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
     if (all || changed()) {
       clearChanged();
+      updateSnapshotMetrics(metricsRecordBuilder);
+    }
+  }
+  
+  public void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder) {
       final Snapshot s = sample.getSnapshot();
       metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc), count.get());
 
@@ -132,6 +137,5 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
           s.get95thPercentile());
       metricsRecordBuilder.addGauge(Interns.info(name + NINETY_NINETH_PERCENTILE_METRIC_NAME, desc),
           s.get99thPercentile());
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/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
new file mode 100644
index 0000000..ac1f497
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableRangeHistogram.java
@@ -0,0 +1,94 @@
+/**
+ * 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.metrics2.lib;
+
+import java.util.concurrent.atomic.AtomicLongArray;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+
+/**
+ * Extended histogram implementation with metric range counters.
+ */
+@InterfaceAudience.Private
+public abstract class MutableRangeHistogram extends MutableHistogram {
+
+  public MutableRangeHistogram(MetricsInfo info) {
+    this(info.name(), info.description());
+  }
+
+  public MutableRangeHistogram(String name, String description) {
+    super(name, description);    
+  }
+  
+  /**
+   * Returns the type of range histogram size or time 
+   */
+  public abstract String getRangeType();
+  
+  /**
+   * Returns the ranges to be counted 
+   */
+  public abstract long[] getRange();
+  
+  /**
+   * Returns the range counts 
+   */
+  public abstract AtomicLongArray getRangeVals();
+
+  @Override
+  public void add(final long val) {
+    super.add(val);
+    updateBand(val);
+  }
+
+  private void updateBand(final long val) {
+    int i;
+    for (i=0; i<getRange().length && val > getRange()[i]; i++);
+    getRangeVals().incrementAndGet(i);
+  }
+  
+  @Override
+  public void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
+    if (all || changed()) {
+      clearChanged();
+      updateSnapshotMetrics(metricsRecordBuilder);
+      updateSnapshotRangeMetrics(metricsRecordBuilder);
+    }
+  }
+  
+  public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder) {
+    long prior = 0;
+    for (int i = 0; i < getRange().length; i++) {
+      long val = getRangeVals().get(i);
+      if (val > 0) {
+        metricsRecordBuilder.addCounter(
+          Interns.info(name + "_" + getRangeType() + "_" + prior + "-" + getRange()[i], desc), val);
+      }
+      prior = getRange()[i];
+    }
+    long val = getRangeVals().get(getRange().length);
+    if (val > 0) {
+      metricsRecordBuilder.addCounter(
+        Interns.info(name + "_" + getRangeType() + "_" + getRange()[getRange().length - 1] + "-inf", desc),
+        getRangeVals().get(getRange().length));
+    }
+  }  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
new file mode 100644
index 0000000..2f1d57a
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableSizeHistogram.java
@@ -0,0 +1,57 @@
+/**
+ * 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.metrics2.lib;
+
+import java.util.concurrent.atomic.AtomicLongArray;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsInfo;
+
+/**
+ * Extended histogram implementation with counters for metric size ranges.
+ */
+@InterfaceAudience.Private
+public class MutableSizeHistogram extends MutableRangeHistogram {
+  private final String rangeType = "SizeRangeCount";
+  private final long[] ranges = {10,100,1000,10000,100000,1000000,10000000,100000000};
+  private final AtomicLongArray rangeVals = new AtomicLongArray(getRange().length+1);
+
+  public MutableSizeHistogram(MetricsInfo info) {
+    this(info.name(), info.description());
+  }
+
+  public MutableSizeHistogram(String name, String description) {
+    super(name, description);
+  }
+
+  @Override
+  public String getRangeType() {
+    return rangeType;
+  }
+
+  @Override
+  public long[] getRange() {
+    return ranges;
+  }
+  
+  @Override
+  public AtomicLongArray getRangeVals() {
+    return rangeVals;
+  }  
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.java
new file mode 100644
index 0000000..32d4fae
--- /dev/null
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/MutableTimeHistogram.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.metrics2.lib;
+
+import java.util.concurrent.atomic.AtomicLongArray;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsInfo;
+
+/**
+ * Extended histogram implementation with counters for metric time ranges.
+ */
+@InterfaceAudience.Private
+public class MutableTimeHistogram extends MutableRangeHistogram {
+  private final String rangeType = "TimeRangeCount";
+  private final long[] ranges =
+      { 1, 3, 10, 30, 100, 300, 1000, 3000, 10000, 30000, 60000, 120000, 300000, 600000 };
+  private final AtomicLongArray rangeVals = new AtomicLongArray(ranges.length+1);
+
+  public MutableTimeHistogram(MetricsInfo info) {
+    this(info.name(), info.description());
+  }
+
+  public MutableTimeHistogram(String name, String description) {
+    super(name, description);
+  }
+
+  @Override
+  public String getRangeType() {
+    return rangeType;
+  }
+
+  @Override
+  public long[] getRange() {
+    return ranges;
+  }
+
+  @Override
+  public AtomicLongArray getRangeVals() {
+    return rangeVals;
+  } 
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
index 42c293b..5503675 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
@@ -205,6 +205,13 @@ public class MetricsAssertHelperImpl implements MetricsAssertHelper {
   }
 
   @Override
+  public boolean checkCounterExists(String name, BaseSource source) {
+    getMetrics(source);
+    String cName = canonicalizeMetricName(name);
+    return (counters.get(cName) != null) ? true : false;
+  }
+  
+  @Override
   public double getGaugeDouble(String name, BaseSource source) {
     getMetrics(source);
     String cName = canonicalizeMetricName(name);
@@ -220,7 +227,6 @@ public class MetricsAssertHelperImpl implements MetricsAssertHelper {
     return gauges.get(cName).longValue();
   }
 
-
   private void reset() {
     tags.clear();
     gauges.clear();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7bfbb6a3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 0a95210..79df5e8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -488,4 +488,71 @@ public class TestRegionServerMetrics {
     admin.close();
     connection.close();
   }
+  
+  @Test
+  public void testRangeCountMetrics() throws Exception {
+    String tableNameString = "testRangeCountMetrics";
+    final long[] timeranges =
+        { 1, 3, 10, 30, 100, 300, 1000, 3000, 10000, 30000, 60000, 120000, 300000, 600000 };
+    final String timeRangeType = "TimeRangeCount";
+    final String timeRangeMetricName = "Mutate";
+    boolean timeRangeCountUpdated = false;
+
+    TableName tName = TableName.valueOf(tableNameString);
+    byte[] cfName = Bytes.toBytes("d");
+    byte[] row = Bytes.toBytes("rk");
+    byte[] qualifier = Bytes.toBytes("qual");
+    byte[] initValue = Bytes.toBytes("Value");
+
+    TEST_UTIL.createTable(tName, cfName);
+
+    Connection connection = TEST_UTIL.getConnection();
+    connection.getTable(tName).close(); // wait for the table to come up.
+
+    // Do a first put to be sure that the connection is established, meta is there and so on.
+    Table table = connection.getTable(tName);
+    Put p = new Put(row);
+    p.addColumn(cfName, qualifier, initValue);
+    table.put(p);
+
+    // do some puts and gets
+    for (int i = 0; i < 10; i++) {
+      table.put(p);
+    }
+
+    Get g = new Get(row);
+    for (int i = 0; i < 10; i++) {
+      table.get(g);
+    }
+
+    metricsRegionServer.getRegionServerWrapper().forceRecompute();
+
+    // Check some time range counters were updated
+    long prior = 0;
+
+    String dynamicMetricName;
+    for (int i = 0; i < timeranges.length; i++) {
+      dynamicMetricName =
+          timeRangeMetricName + "_" + timeRangeType + "_" + prior + "-" + timeranges[i];
+      if (metricsHelper.checkCounterExists(dynamicMetricName, serverSource)) {
+        long count = metricsHelper.getCounter(dynamicMetricName, serverSource);
+        if (count > 0) {
+          timeRangeCountUpdated = true;
+          break;
+        }
+      }
+      prior = timeranges[i];
+    }
+    dynamicMetricName =
+        timeRangeMetricName + "_" + timeRangeType + "_" + timeranges[timeranges.length - 1] + "-inf";
+    if (metricsHelper.checkCounterExists(dynamicMetricName, serverSource)) {
+      long count = metricsHelper.getCounter(dynamicMetricName, serverSource);
+      if (count > 0) {
+        timeRangeCountUpdated = true;
+      }
+    }
+    assertEquals(true, timeRangeCountUpdated);
+
+    table.close();
+  }
 }