You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2012/11/07 00:22:09 UTC

svn commit: r1406396 [4/6] - in /hbase/trunk: dev-support/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/ hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/met...

Modified: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java (original)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/test/MetricsAssertHelperImpl.java Tue Nov  6 23:22:01 2012
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.hbase.test;
 
-import org.apache.hadoop.hbase.metrics.BaseMetricsSource;
-import org.apache.hadoop.hbase.metrics.BaseMetricsSourceImpl;
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.AbstractMetric;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsInfo;
@@ -129,68 +129,68 @@ public class MetricsAssertHelperImpl imp
   }
 
   @Override
-  public void assertTag(String name, String expected, BaseMetricsSource source) {
+  public void assertTag(String name, String expected, BaseSource source) {
     getMetrics(source);
     String cName = canonicalizeMetricName(name);
     assertEquals("Tags should be equal", expected, tags.get(cName));
   }
 
   @Override
-  public void assertGauge(String name, long expected, BaseMetricsSource source) {
+  public void assertGauge(String name, long expected, BaseSource source) {
     long found = getGaugeLong(name, source);
     assertEquals("Metrics Should be equal", (long) Long.valueOf(expected), found);
   }
 
   @Override
-  public void assertGaugeGt(String name, long expected, BaseMetricsSource source) {
+  public void assertGaugeGt(String name, long expected, BaseSource source) {
     double found = getGaugeDouble(name, source);
     assertTrue(name + " (" + found + ") should be greater than " + expected, found > expected);
   }
 
   @Override
-  public void assertGaugeLt(String name, long expected, BaseMetricsSource source) {
+  public void assertGaugeLt(String name, long expected, BaseSource source) {
     double found = getGaugeDouble(name, source);
     assertTrue(name + "(" + found + ") should be less than " + expected, found < expected);
   }
 
   @Override
-  public void assertGauge(String name, double expected, BaseMetricsSource source) {
+  public void assertGauge(String name, double expected, BaseSource source) {
     double found = getGaugeDouble(name, source);
-    assertEquals("Metrics Should be equal", (double) Double.valueOf(expected), found);
+    assertEquals("Metrics Should be equal", (double) Double.valueOf(expected), found, 0.01);
   }
 
   @Override
-  public void assertGaugeGt(String name, double expected, BaseMetricsSource source) {
+  public void assertGaugeGt(String name, double expected, BaseSource source) {
     double found = getGaugeDouble(name, source);
     assertTrue(name + "(" + found + ") should be greater than " + expected, found > expected);
   }
 
   @Override
-  public void assertGaugeLt(String name, double expected, BaseMetricsSource source) {
+  public void assertGaugeLt(String name, double expected, BaseSource source) {
     double found = getGaugeDouble(name, source);
     assertTrue(name + "(" + found + ") should be less than " + expected, found < expected);
   }
 
   @Override
-  public void assertCounter(String name, long expected, BaseMetricsSource source) {
+  public void assertCounter(String name, long expected, BaseSource source) {
     long found = getCounter(name, source);
     assertEquals("Metrics Counters should be equal", (long) Long.valueOf(expected), found);
   }
 
   @Override
-  public void assertCounterGt(String name, long expected, BaseMetricsSource source) {
+  public void assertCounterGt(String name, long expected, BaseSource source) {
     long found = getCounter(name, source);
     assertTrue(name + " (" + found + ") should be greater than " + expected, found > expected);
   }
 
   @Override
-  public void assertCounterLt(String name, long expected, BaseMetricsSource source) {
+  public void assertCounterLt(String name, long expected, BaseSource source) {
     long found = getCounter(name, source);
     assertTrue(name + "(" + found + ") should be less than " + expected, found < expected);
   }
 
   @Override
-  public long getCounter(String name, BaseMetricsSource source) {
+  public long getCounter(String name, BaseSource source) {
     getMetrics(source);
     String cName = canonicalizeMetricName(name);
     assertNotNull(counters.get(cName));
@@ -198,7 +198,7 @@ public class MetricsAssertHelperImpl imp
   }
 
   @Override
-  public double getGaugeDouble(String name, BaseMetricsSource source) {
+  public double getGaugeDouble(String name, BaseSource source) {
     getMetrics(source);
     String cName = canonicalizeMetricName(name);
     assertNotNull(gauges.get(cName));
@@ -206,7 +206,7 @@ public class MetricsAssertHelperImpl imp
   }
 
   @Override
-  public long getGaugeLong(String name, BaseMetricsSource source) {
+  public long getGaugeLong(String name, BaseSource source) {
     getMetrics(source);
     String cName = canonicalizeMetricName(name);
     assertNotNull(gauges.get(cName));
@@ -220,12 +220,12 @@ public class MetricsAssertHelperImpl imp
     counters.clear();
   }
 
-  private void getMetrics(BaseMetricsSource source) {
+  private void getMetrics(BaseSource source) {
     reset();
-    if (!(source instanceof BaseMetricsSourceImpl)) {
+    if (!(source instanceof BaseSourceImpl)) {
       assertTrue(false);
     }
-    BaseMetricsSourceImpl impl = (BaseMetricsSourceImpl) source;
+    BaseSourceImpl impl = (BaseSourceImpl) source;
 
     impl.getMetrics(new MockMetricsBuilder(), true);
 

Added: hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java (added)
+++ hbase/trunk/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/thrift/TestMetricsThriftServerSourceFactoryImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,55 @@
+/**
+ * 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.thrift;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory;
+import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactoryImpl;
+import org.junit.Test;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ *   Test for hadoop 2's version of MetricsThriftServerSourceFactory
+ */
+public class TestMetricsThriftServerSourceFactoryImpl {
+
+  @Test
+  public void testCompatabilityRegistered() throws Exception {
+    assertNotNull(CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class));
+    assertTrue(CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class) instanceof MetricsThriftServerSourceFactoryImpl);
+  }
+
+  @Test
+  public void testCreateThriftOneSource() throws Exception {
+    //Make sure that the factory gives back a singleton.
+    assertSame(new MetricsThriftServerSourceFactoryImpl().createThriftOneSource(),
+        new MetricsThriftServerSourceFactoryImpl().createThriftOneSource());
+
+  }
+
+  @Test
+  public void testCreateThriftTwoSource() throws Exception {
+    //Make sure that the factory gives back a singleton.
+    assertSame(new MetricsThriftServerSourceFactoryImpl().createThriftTwoSource(),
+        new MetricsThriftServerSourceFactoryImpl().createThriftTwoSource());
+  }
+}

Modified: hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon (original)
+++ hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon Tue Nov  6 23:22:01 2012
@@ -24,7 +24,6 @@ String format = "html";
 <%import>
 java.util.*;
 org.apache.hadoop.hbase.regionserver.HRegionServer;
-org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
 org.apache.hadoop.hbase.util.Bytes;
 org.apache.hadoop.hbase.HRegionInfo;
 org.apache.hadoop.hbase.ServerName;
@@ -38,7 +37,6 @@ org.apache.hadoop.hbase.protobuf.generat
   <%java return; %>
 </%if>
 <%java>
-  RegionServerMetrics metrics = regionServer.getMetrics();
   ServerInfo serverInfo = ProtobufUtil.getServerInfo(regionServer);
   ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
   List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer);
@@ -98,7 +96,7 @@ org.apache.hadoop.hbase.protobuf.generat
     </div>
 
     <h2>Server Metrics</h2>
-    <& ServerMetricsTmpl; metrics = metrics; &>
+    <& ServerMetricsTmpl; mWrap = regionServer.getMetrics().getRegionServerWrapper(); &>
 
     <& ../common/TaskMonitorTmpl; filter = filter &>
 

Modified: hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon (original)
+++ hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon Tue Nov  6 23:22:01 2012
@@ -23,7 +23,6 @@
 <%import>
         java.util.*;
         org.apache.hadoop.hbase.regionserver.HRegionServer;
-        org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
         org.apache.hadoop.hbase.util.Bytes;
         org.apache.hadoop.hbase.HRegionInfo;
         org.apache.hadoop.hbase.ServerName;

Modified: hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon (original)
+++ hbase/trunk/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon Tue Nov  6 23:22:01 2012
@@ -17,12 +17,12 @@ See the License for the specific languag
 limitations under the License.
 </%doc>
 <%args>
-    RegionServerMetrics metrics;
+MetricsRegionServerWrapper mWrap;
 </%args>
 <%import>
 java.util.*;
 org.apache.hadoop.hbase.regionserver.HRegionServer;
-org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
+org.apache.hadoop.hbase.regionserver.MetricsRegionServerWrapper;
 org.apache.hadoop.hbase.util.Bytes;
 org.apache.hadoop.hbase.HRegionInfo;
 org.apache.hadoop.hbase.ServerName;
@@ -42,36 +42,32 @@ java.lang.management.ManagementFactory;
         <li class=""><a href="#storeStats" data-toggle="tab">Storefiles</a></li>
         <li class=""><a href="#queueStats" data-toggle="tab">Queues</a></li>
         <li class=""><a href="#blockCacheStats" data-toggle="tab">Block Cache</a></li>
-        <li class=""><a href="#latencyStats" data-toggle="tab">Latency</a></li>
     </ul>
     <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
         <div class="tab-pane active" id="baseStats">
-            <& baseStats; metrics = metrics; &>
+            <& baseStats; mWrap = mWrap &>
         </div>
         <div class="tab-pane" id="memoryStats">
-            <& memoryStats; metrics = metrics; &>
+            <& memoryStats; mWrap = mWrap &>
         </div>
         <div class="tab-pane" id="requestStats">
-            <& requestStats; metrics = metrics; &>
+            <& requestStats; mWrap = mWrap &>
         </div>
         <div class="tab-pane" id="storeStats">
-            <& storeStats; metrics = metrics; &>
+            <& storeStats; mWrap = mWrap &>
         </div>
         <div class="tab-pane" id="queueStats">
-            <& queueStats; metrics = metrics; &>
+            <& queueStats; mWrap = mWrap  &>
         </div>
         <div class="tab-pane" id="blockCacheStats">
-            <& blockCacheStats; metrics = metrics; &>
-        </div>
-        <div class="tab-pane" id="latencyStats">
-            <& latencyStats; metrics = metrics; &>
+            <& blockCacheStats; mWrap = mWrap &>
         </div>
     </div>
 </div>
 
 <%def baseStats>
 <%args>
-    RegionServerMetrics metrics;
+    MetricsRegionServerWrapper mWrap;
 </%args>
 <table class="table table-striped">
     <tr>
@@ -82,17 +78,17 @@ java.lang.management.ManagementFactory;
         <th>Slow HLog Append Count</th>
     </tr>
     <tr>
-        <td><% metrics.requests.getPreviousIntervalValue() %></td>
-        <td><% metrics.regions.get() %></td>
-        <td><% metrics.hdfsBlocksLocalityIndex.get() %></td>
-        <td><% metrics.slowHLogAppendCount.get() %></td>
+        <td><% mWrap.getRequestsPerSecond() %></td>
+        <td><% mWrap.getNumOnlineRegions() %></td>
+        <td><% mWrap.getPercentFileLocal() %></td>
+        <td><% 0 %></td>
     </tr>
 </table>
 </%def>
 
 <%def memoryStats>
 <%args>
-        RegionServerMetrics metrics;
+MetricsRegionServerWrapper mWrap;
 </%args>
 <table class="table table-striped">
 <tr>
@@ -104,19 +100,19 @@ java.lang.management.ManagementFactory;
 </tr>
 <tr>
     <td>
-        <% ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getUsed() / (1024*1024) %>MB
+        <% ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getUsed() %>
     </td>
     <td>
-        <% ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax() / (1024*1024) %>MB
+        <% ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax()%>
     </td>
-    <td><% metrics.memstoreSizeMB.get()%>MB</td>
+    <td><% mWrap.getMemstoreSize() %></td>
 </tr>
 </table>
 </%def>
 
 <%def storeStats>
 <%args>
-    RegionServerMetrics metrics;
+MetricsRegionServerWrapper mWrap;
 </%args>
 <table class="table table-striped">
 <tr>
@@ -128,11 +124,11 @@ java.lang.management.ManagementFactory;
     <th>Bloom Size</th>
 </tr>
 <tr>
-    <td><% metrics.stores.get() %></td>
-    <td><% metrics.storefiles.get() %></td>
-    <td><% metrics.rootIndexSizeKB.get() %>KB</td>
-    <td><% metrics.totalStaticIndexSizeKB.get() %>KB</td>
-    <td><% metrics.totalStaticBloomSizeKB.get() %>KB</td>
+    <td><% mWrap.getNumStores() %></td>
+    <td><% mWrap.getNumStoreFiles() %></td>
+    <td><% mWrap.getStoreFileIndexSize() %></td>
+    <td><% mWrap.getTotalStaticIndexSize() %></td>
+    <td><% mWrap.getTotalStaticBloomSize() %></td>
 </tr>
 </table>
 </%def>
@@ -140,8 +136,8 @@ java.lang.management.ManagementFactory;
 
 <%def requestStats>
 <%args>
-        RegionServerMetrics metrics;
-        </%args>
+MetricsRegionServerWrapper mWrap;
+</%args>
 <table class="table table-striped">
 <tr>
     <th>Request Per Second</th>
@@ -149,17 +145,17 @@ java.lang.management.ManagementFactory;
     <th>Write Request Count</th>
 </tr>
 <tr>
-    <td><% metrics.requests.getPreviousIntervalValue() %></td>
-    <td><% metrics.readRequestsCount.get() %></td>
-    <td><% metrics.writeRequestsCount.get() %>KB</td>
+    <td><% mWrap.getRequestsPerSecond() %></td>
+    <td><% mWrap.getReadRequestsCount() %></td>
+    <td><% mWrap.getWriteRequestsCount() %>KB</td>
 </tr>
 </table>
 </%def>
 
 <%def queueStats>
 <%args>
-        RegionServerMetrics metrics;
-        </%args>
+MetricsRegionServerWrapper mWrap;
+</%args>
 <table class="table table-striped">
 <tr>
     <th>Compaction queue size</th>
@@ -167,8 +163,8 @@ java.lang.management.ManagementFactory;
 
 </tr>
 <tr>
-    <td><% metrics.compactionQueueSize.get() %></td>
-    <td><% metrics.flushQueueSize.get() %>KB</td>
+    <td><% mWrap.getCompactionQueueSize() %></td>
+    <td><% mWrap.getFlushQueueSize() %></td>
 </tr>
 </table>
 </%def>
@@ -176,8 +172,8 @@ java.lang.management.ManagementFactory;
 
 <%def blockCacheStats>
 <%args>
-        RegionServerMetrics metrics;
-        </%args>
+MetricsRegionServerWrapper mWrap;
+</%args>
 <table class="table table-striped">
 <tr>
     <th>Cache Size</th>
@@ -190,57 +186,13 @@ java.lang.management.ManagementFactory;
 
 </tr>
 <tr>
-    <td><% metrics.blockCacheSize.get() / (1024*1024) %>MB</td>
-    <td><% metrics.blockCacheFree.get() / (1024 * 1024) %>MB</td>
-    <td><% metrics.blockCacheCount.get()%></td>
-    <td><% metrics.blockCacheHitCount.get()%></td>
-    <td><% metrics.blockCacheMissCount.get()%></td>
-    <td><% metrics.blockCacheHitRatio.get()%>%</td>
-    <td><% metrics.blockCacheEvictedCount.get()%></td>
-</tr>
-</table>
-</%def>
-
-<%def latencyStats>
-<%args>
-        RegionServerMetrics metrics;
-        </%args>
-<table class="table table-striped">
-<tr>
-    <th>Operation</th>
-    <th>Count</th>
-    <th>Mean</th>
-    <th>Median</th>
-    <th>75th</th>
-    <th>95th</th>
-    <th>99th</th>
-    <th>99.9th</th>
-
+    <td><% mWrap.getBlockCacheSize()%></td>
+    <td><% mWrap.getBlockCacheFreeSize()%></td>
+    <td><% mWrap.getBlockCacheCount() %></td>
+    <td><% mWrap.getBlockCacheHitCount() %></td>
+    <td><% mWrap.getBlockCacheMissCount() %></td>
+    <td><% mWrap.getBlockCacheHitPercent() %>%</td>
+    <td><% mWrap.getBlockCacheEvictedCount() %></td>
 </tr>
-
-    <& histogramRow; op ="FS Read"; histo = metrics.fsReadLatencyHistogram &>
-    <& histogramRow; op ="FS PRead"; histo = metrics.fsPreadLatencyHistogram &>
-    <& histogramRow; op ="FS Write"; histo = metrics.fsWriteLatencyHistogram &>
-
 </table>
-</%def>
-
-<%def histogramRow>
-<%args>
-        String op;
-        MetricsHistogram histo;
-</%args>
-<%java>
-    Snapshot s = histo.getSnapshot();
-</%java>
-<tr>
-    <td><% op %></td>
-    <td><% histo.getCount()%></td>
-    <td><% String.format("%10.2f", histo.getMean()) %></td>
-    <td><% String.format("%10.2f", s.getMedian()) %></td>
-    <td><% String.format("%10.2f", s.get75thPercentile()) %></td>
-    <td><% String.format("%10.2f", s.get95thPercentile()) %></td>
-    <td><% String.format("%10.2f", s.get99thPercentile()) %></td>
-    <td><% String.format("%10.2f", s.get999thPercentile())%></td>
-</tr>
-</%def>
+</%def>
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileReader.java Tue Nov  6 23:22:01 2012
@@ -30,15 +30,13 @@ import org.apache.hadoop.hbase.fs.HFileS
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.io.RawComparator;
 
 /**
  * Common functionality needed by all versions of {@link HFile} readers.
  */
 @InterfaceAudience.Private
-public abstract class AbstractHFileReader extends SchemaConfigured
-    implements HFile.Reader {
+public abstract class AbstractHFileReader implements HFile.Reader {
 
   /** Filesystem-level block reader for this HFile format version. */
   protected HFileBlock.FSReader fsBlockReader;
@@ -119,7 +117,6 @@ public abstract class AbstractHFileReade
       final long fileSize,
       final boolean closeIStream,
       final CacheConfig cacheConf, final HFileSystem hfs) {
-    super(null, path);
     this.trailer = trailer;
     this.compressAlgo = trailer.getCompressionCodec();
     this.cacheConf = cacheConf;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AbstractHFileWriter.java Tue Nov  6 23:22:01 2012
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.io.RawComparator;
@@ -44,8 +43,7 @@ import org.apache.hadoop.io.Writable;
  * Common functionality needed by all versions of {@link HFile} writers.
  */
 @InterfaceAudience.Private
-public abstract class AbstractHFileWriter extends SchemaConfigured
-    implements HFile.Writer {
+public abstract class AbstractHFileWriter implements HFile.Writer {
 
   /** Key previously appended. Becomes the last key in the file. */
   protected byte[] lastKeyBuffer = null;
@@ -116,7 +114,6 @@ public abstract class AbstractHFileWrite
       Compression.Algorithm compressAlgo,
       HFileDataBlockEncoder dataBlockEncoder,
       KeyComparator comparator) {
-    super(null, path);
     this.outputStream = outputStream;
     this.path = path;
     this.name = path != null ? path.getName() : outputStream.toString();

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/Cacheable.java Tue Nov  6 23:22:01 2012
@@ -23,7 +23,6 @@ import java.nio.ByteBuffer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 
 /**
  * Cacheable is an interface that allows for an object to be cached. If using an
@@ -57,14 +56,4 @@ public interface Cacheable extends HeapS
    */
   public CacheableDeserializer<Cacheable> getDeserializer();
 
-  /**
-   * @return the block type of this cached HFile block
-   */
-  public BlockType getBlockType();
-
-  /**
-   * @return the metrics object identified by table and column family
-   */
-  public SchemaMetrics getSchemaMetrics();
-
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java Tue Nov  6 23:22:01 2012
@@ -49,8 +49,6 @@ import org.apache.hadoop.hbase.fs.HFileS
 import org.apache.hadoop.hbase.io.HbaseMapWritable;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
@@ -284,8 +282,6 @@ public class HFile {
     /** @return the path to this {@link HFile} */
     Path getPath();
 
-    String getColumnFamilyName();
-
     void appendMetaBlock(String bloomFilterMetaKey, Writable metaWriter);
 
     /**
@@ -431,7 +427,6 @@ public class HFile {
    */
   public static final WriterFactory getWriterFactory(Configuration conf,
       CacheConfig cacheConf) {
-    SchemaMetrics.configureGlobally(conf);
     int version = getFormatVersion(conf);
     switch (version) {
     case 1:
@@ -453,8 +448,7 @@ public class HFile {
   }
 
   /** An interface used by clients to open and iterate an {@link HFile}. */
-  public interface Reader extends Closeable, CachingBlockReader,
-      SchemaAware {
+  public interface Reader extends Closeable, CachingBlockReader {
     /**
      * Returns this reader's "name". Usually the last component of the path.
      * Needs to be constant as the file is being moved to support caching on
@@ -462,8 +456,6 @@ public class HFile {
      */
     String getName();
 
-    String getColumnFamilyName();
-
     RawComparator<byte []> getComparator();
 
     HFileScanner getScanner(boolean cacheBlocks,

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java Tue Nov  6 23:22:01 2012
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.io.encodi
 import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
 import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
 import org.apache.hadoop.hbase.regionserver.MemStore;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -85,7 +84,7 @@ import com.google.common.base.Preconditi
  * except that the data section is always uncompressed in the cache.
  */
 @InterfaceAudience.Private
-public class HFileBlock extends SchemaConfigured implements Cacheable {
+public class HFileBlock implements Cacheable {
 
   /** Minor versions starting with this number have hbase checksums */
   static final int MINOR_VERSION_WITH_CHECKSUM = 1;
@@ -539,8 +538,7 @@ public class HFileBlock extends SchemaCo
   @Override
   public long heapSize() {
     long size = ClassSize.align(
-        // Base class size, including object overhead.
-        SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE +
+        ClassSize.OBJECT +
         // Block type and byte buffer references
         2 * ClassSize.REFERENCE +
         // On-disk size, uncompressed size, and next block's on-disk size

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java Tue Nov  6 23:22:01 2012
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.HFile.CachingBlockReader;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.CompoundBloomFilterWriter;
@@ -719,8 +718,7 @@ public class HFileBlockIndex {
    * index. However, in most practical cases we will only have leaf-level
    * blocks and the root index, or just the root index.
    */
-  public static class BlockIndexWriter extends SchemaConfigured
-      implements InlineBlockWriter {
+  public static class BlockIndexWriter implements InlineBlockWriter {
     /**
      * While the index is being written, this represents the current block
      * index referencing all leaf blocks, with one exception. If the file is
@@ -954,7 +952,6 @@ public class HFileBlockIndex {
 
       if (blockCache != null) {
         HFileBlock blockForCaching = blockWriter.getBlockForCaching();
-        passSchemaMetricsTo(blockForCaching);
         blockCache.cacheBlock(new BlockCacheKey(nameForCaching,
             beginOffset, DataBlockEncoding.NONE, 
             blockForCaching.getBlockType()), blockForCaching);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileDataBlockEncoderImpl.java Tue Nov  6 23:22:01 2012
@@ -243,7 +243,6 @@ public class HFileDataBlockEncoderImpl i
         includesMemstoreTS, block.getMinorVersion(),
         block.getBytesPerChecksum(), block.getChecksumType(),
         block.getOnDiskDataSizeWithHeader());
-    block.passSchemaMetricsTo(encodedBlock);
     return encodedBlock;
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java Tue Nov  6 23:22:01 2012
@@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
 import org.apache.hadoop.hbase.util.ByteBloomFilter;
@@ -174,7 +173,6 @@ public class HFilePrettyPrinter {
         conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
     conf.set("fs.default.name",
         conf.get(org.apache.hadoop.hbase.HConstants.HBASE_DIR));
-    SchemaMetrics.configureGlobally(conf);
     try {
       if (!parseOptions(args))
         return 1;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV1.java Tue Nov  6 23:22:01 2012
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.io.encodi
 import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.RawComparator;
@@ -235,8 +234,6 @@ public class HFileReaderV1 extends Abstr
               cacheConf.shouldCacheBlockOnRead(effectiveCategory));
         if (cachedBlock != null) {
           cacheHits.incrementAndGet();
-          getSchemaMetrics().updateOnCacheHit(effectiveCategory,
-              SchemaMetrics.NO_COMPACTION);
           return cachedBlock.getBufferWithoutHeader();
         }
         // Cache Miss, please load.
@@ -245,13 +242,10 @@ public class HFileReaderV1 extends Abstr
       HFileBlock hfileBlock = fsBlockReader.readBlockData(offset,
           nextOffset - offset, metaBlockIndexReader.getRootBlockDataSize(block),
           true);
-      passSchemaMetricsTo(hfileBlock);
       hfileBlock.expectType(BlockType.META);
 
       final long delta = System.nanoTime() - startTimeNs;
       HFile.offerReadLatency(delta, true);
-      getSchemaMetrics().updateOnCacheMiss(effectiveCategory,
-          SchemaMetrics.NO_COMPACTION, delta);
 
       // Cache the block
       if (cacheBlock && cacheConf.shouldCacheBlockOnRead(effectiveCategory)) {
@@ -300,8 +294,6 @@ public class HFileReaderV1 extends Abstr
               cacheConf.shouldCacheDataOnRead());
         if (cachedBlock != null) {
           cacheHits.incrementAndGet();
-          getSchemaMetrics().updateOnCacheHit(
-              cachedBlock.getBlockType().getCategory(), isCompaction);
           return cachedBlock.getBufferWithoutHeader();
         }
         // Carry on, please load.
@@ -323,13 +315,10 @@ public class HFileReaderV1 extends Abstr
 
       HFileBlock hfileBlock = fsBlockReader.readBlockData(offset, nextOffset
           - offset, dataBlockIndexReader.getRootBlockDataSize(block), pread);
-      passSchemaMetricsTo(hfileBlock);
       hfileBlock.expectType(BlockType.DATA);
 
       final long delta = System.nanoTime() - startTimeNs;
       HFile.offerReadLatency(delta, pread);
-      getSchemaMetrics().updateOnCacheMiss(BlockCategory.DATA, isCompaction,
-          delta);
 
       // Cache the block
       if (cacheBlock && cacheConf.shouldCacheBlockOnRead(

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderV2.java Tue Nov  6 23:22:01 2012
@@ -226,7 +226,6 @@ public class HFileReaderV2 extends Abstr
           // Return a distinct 'shallow copy' of the block,
           // so pos does not get messed by the scanner
           cacheHits.incrementAndGet();
-          getSchemaMetrics().updateOnCacheHit(BlockCategory.META, false);
           return cachedBlock.getBufferWithoutHeader();
         }
         // Cache Miss, please load.
@@ -234,11 +233,9 @@ public class HFileReaderV2 extends Abstr
 
       HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset,
           blockSize, -1, true);
-      passSchemaMetricsTo(metaBlock);
 
       final long delta = System.nanoTime() - startTimeNs;
       HFile.offerReadLatency(delta, true);
-      getSchemaMetrics().updateOnCacheMiss(BlockCategory.META, false, delta);
 
       // Cache the block
       if (cacheBlock) {
@@ -302,7 +299,6 @@ public class HFileReaderV2 extends Abstr
               cachedBlock.getBlockType().getCategory();
           cacheHits.incrementAndGet();
 
-          getSchemaMetrics().updateOnCacheHit(blockCategory, isCompaction);
 
           if (cachedBlock.getBlockType() == BlockType.DATA) {
             HFile.dataBlockReadCnt.incrementAndGet();
@@ -331,12 +327,10 @@ public class HFileReaderV2 extends Abstr
       hfileBlock = dataBlockEncoder.diskToCacheFormat(hfileBlock,
           isCompaction);
       validateBlockType(hfileBlock, expectedBlockType);
-      passSchemaMetricsTo(hfileBlock);
       BlockCategory blockCategory = hfileBlock.getBlockType().getCategory();
 
       final long delta = System.nanoTime() - startTimeNs;
       HFile.offerReadLatency(delta, pread);
-      getSchemaMetrics().updateOnCacheMiss(blockCategory, isCompaction, delta);
 
       // Cache the block if necessary
       if (cacheBlock && cacheConf.shouldCacheBlockOnRead(

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV1.java Tue Nov  6 23:22:01 2012
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.io.encodi
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.regionserver.MemStore;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -109,7 +108,6 @@ public class HFileWriterV1 extends Abstr
       final KeyComparator comparator) throws IOException {
     super(cacheConf, ostream == null ? createOutputStream(conf, fs, path) : ostream, path,
         blockSize, compress, blockEncoder, comparator);
-    SchemaMetrics.configureGlobally(conf);
   }
 
   /**
@@ -158,7 +156,6 @@ public class HFileWriterV1 extends Abstr
           HFileBlock.HEADER_SIZE_NO_CHECKSUM);       // onDiskDataSizeWithHeader
 
       block = blockEncoder.diskToCacheFormat(block, false);
-      passSchemaMetricsTo(block);
       cacheConf.getBlockCache().cacheBlock(
           new BlockCacheKey(name, blockBegin, DataBlockEncoding.NONE,
               block.getBlockType()), block);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterV2.java Tue Nov  6 23:22:01 2012
@@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.KeyValue.
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock.BlockWritable;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -114,7 +113,6 @@ public class HFileWriterV2 extends Abstr
     super(cacheConf,
         ostream == null ? createOutputStream(conf, fs, path) : ostream,
         path, blockSize, compressAlgo, blockEncoder, comparator);
-    SchemaMetrics.configureGlobally(conf);
     this.checksumType = checksumType;
     this.bytesPerChecksum = bytesPerChecksum;
     finishInit(conf);
@@ -141,16 +139,6 @@ public class HFileWriterV2 extends Abstr
     // Meta data block index writer
     metaBlockIndexWriter = new HFileBlockIndex.BlockIndexWriter();
     LOG.debug("Initialized with " + cacheConf);
-
-    if (isSchemaConfigured()) {
-      schemaConfigurationChanged();
-    }
-  }
-
-  @Override
-  protected void schemaConfigurationChanged() {
-    passSchemaMetricsTo(dataBlockIndexWriter);
-    passSchemaMetricsTo(metaBlockIndexWriter);
   }
 
   /**
@@ -227,7 +215,6 @@ public class HFileWriterV2 extends Abstr
     final boolean isCompaction = false;
     HFileBlock cacheFormatBlock = blockEncoder.diskToCacheFormat(
         fsBlockWriter.getBlockForCaching(), isCompaction);
-    passSchemaMetricsTo(cacheFormatBlock);
     cacheConf.getBlockCache().cacheBlock(
         new BlockCacheKey(name, offset, blockEncoder.getEncodingInCache(),
             cacheFormatBlock.getBlockType()), cacheFormatBlock);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java Tue Nov  6 23:22:01 2012
@@ -44,7 +44,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -326,12 +325,6 @@ public class LruBlockCache implements Bl
     if (evict) {
       heapsize *= -1;
     }
-    Cacheable cachedBlock = cb.getBuffer();
-    SchemaMetrics schemaMetrics = cachedBlock.getSchemaMetrics();
-    if (schemaMetrics != null) {
-      schemaMetrics.updateOnCachePutOrEvict(
-          cachedBlock.getBlockType().getCategory(), heapsize, evict);
-    }
     return size.addAndGet(heapsize);
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java Tue Nov  6 23:22:01 2012
@@ -65,7 +65,6 @@ import org.apache.hadoop.hbase.master.ha
 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
 import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
 import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
-import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
 import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
@@ -152,7 +151,7 @@ public class AssignmentManager extends Z
       EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED });
 
   // metrics instance to send metrics for RITs
-  MasterMetrics masterMetrics;
+  MetricsMaster metricsMaster;
 
   private final RegionStates regionStates;
 
@@ -176,7 +175,7 @@ public class AssignmentManager extends Z
    */
   public AssignmentManager(Server server, ServerManager serverManager,
       CatalogTracker catalogTracker, final LoadBalancer balancer,
-      final ExecutorService service, MasterMetrics metrics) throws KeeperException, IOException {
+      final ExecutorService service, MetricsMaster metricsMaster) throws KeeperException, IOException {
     super(server.getZooKeeper());
     this.server = server;
     this.serverManager = serverManager;
@@ -200,7 +199,7 @@ public class AssignmentManager extends Z
     int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
     this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
       maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("hbase-am"));
-    this.masterMetrics = metrics;// can be null only with tests.
+    this.metricsMaster = metricsMaster;// can be null only with tests.
     this.regionStates = new RegionStates(server, serverManager);
 
     int workers = conf.getInt("hbase.assignment.zkevent.workers", 5);
@@ -2343,10 +2342,10 @@ public class AssignmentManager extends Z
         oldestRITTime = ritTime;
       }
     }
-    if (this.masterMetrics != null) {
-      this.masterMetrics.updateRITOldestAge(oldestRITTime);
-      this.masterMetrics.updateRITCount(totalRITs);
-      this.masterMetrics.updateRITCountOverThreshold(totalRITsOverThreshold);
+    if (this.metricsMaster != null) {
+      this.metricsMaster.updateRITOldestAge(oldestRITTime);
+      this.metricsMaster.updateRITCount(totalRITs);
+      this.metricsMaster.updateRITCountOverThreshold(totalRITsOverThreshold);
     }
   }
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Tue Nov  6 23:22:01 2012
@@ -47,6 +47,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Chore;
+import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DeserializationException;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -96,8 +97,6 @@ import org.apache.hadoop.hbase.master.ha
 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
 import org.apache.hadoop.hbase.master.handler.TableEventHandler;
 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
-import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
-import org.apache.hadoop.hbase.master.metrics.MasterMetricsWrapperImpl;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
@@ -247,7 +246,7 @@ Server {
   private final InetSocketAddress isa;
 
   // Metrics for the HMaster
-  private final MasterMetrics metrics;
+  private final MetricsMaster metricsMaster;
   // file system manager for the master FS operations
   private MasterFileSystem fileSystemManager;
 
@@ -383,7 +382,7 @@ Server {
     //should we check the compression codec type at master side, default true, HBASE-6370
     this.masterCheckCompression = conf.getBoolean("hbase.master.check.compression", true);
 
-    this.metrics = new MasterMetrics( new MasterMetricsWrapperImpl(this));
+    this.metricsMaster = new MetricsMaster( new MetricsMasterWrapperImpl(this));
   }
 
   /**
@@ -413,8 +412,8 @@ Server {
 
   }
 
-  MasterMetrics getMetrics() {
-    return metrics;
+  MetricsMaster getMetrics() {
+    return metricsMaster;
   }
 
   /**
@@ -523,7 +522,7 @@ Server {
     this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
     this.loadBalancerTracker.start();
     this.assignmentManager = new AssignmentManager(this, serverManager,
-      this.catalogTracker, this.balancer, this.executorService, this.metrics);
+      this.catalogTracker, this.balancer, this.executorService, this.metricsMaster);
     zooKeeper.registerListenerFirst(assignmentManager);
 
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
@@ -627,7 +626,7 @@ Server {
     status.setStatus("Initializing Master file system");
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
-    this.fileSystemManager = new MasterFileSystem(this, this, metrics, masterRecovery);
+    this.fileSystemManager = new MasterFileSystem(this, this, metricsMaster, masterRecovery);
 
     this.tableDescriptors =
       new FSTableDescriptors(this.fileSystemManager.getFileSystem(),
@@ -1182,9 +1181,9 @@ Server {
     try {
       HBaseProtos.ServerLoad sl = request.getLoad();
       this.serverManager.regionServerReport(ProtobufUtil.toServerName(request.getServer()), new ServerLoad(sl));
-      if (sl != null && this.metrics != null) {
+      if (sl != null && this.metricsMaster != null) {
         // Up our metrics.
-        this.metrics.incrementRequests(sl.getTotalNumberOfRequests());
+        this.metricsMaster.incrementRequests(sl.getTotalNumberOfRequests());
       }
     } catch (IOException ioe) {
       throw new ServiceException(ioe);
@@ -1834,7 +1833,14 @@ Server {
   }
 
   public String getClusterId() {
-    return fileSystemManager.getClusterId().toString();
+    if (fileSystemManager == null) {
+      return "";
+    }
+    ClusterId id = fileSystemManager.getClusterId();
+    if (id == null) {
+      return "";
+    }
+    return id.toString();
   }
 
   /**
@@ -2232,7 +2238,15 @@ Server {
    * @return the average load
    */
   public double getAverageLoad() {
-    return this.assignmentManager.getRegionStates().getAverageLoad();
+    if (this.assignmentManager == null) {
+      return 0;
+    }
+
+    RegionStates regionStates = this.assignmentManager.getRegionStates();
+    if (regionStates == null) {
+      return 0;
+    }
+    return regionStates.getAverageLoad();
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java Tue Nov  6 23:22:01 2012
@@ -44,10 +44,8 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.fs.HFileSystem;
-import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
-import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
@@ -69,7 +67,7 @@ public class MasterFileSystem {
   // master status
   Server master;
   // metrics for master
-  MasterMetrics metrics;
+  MetricsMaster metricsMaster;
   // Persisted unique cluster ID
   private ClusterId clusterId;
   // Keep around for convenience.
@@ -87,12 +85,12 @@ public class MasterFileSystem {
   private final MasterServices services;
 
   public MasterFileSystem(Server master, MasterServices services,
-      MasterMetrics metrics, boolean masterRecovery)
+      MetricsMaster metricsMaster, boolean masterRecovery)
   throws IOException {
     this.conf = master.getConfiguration();
     this.master = master;
     this.services = services;
-    this.metrics = metrics;
+    this.metricsMaster = metricsMaster;
     // Set filesystem to be that of this.rootdir else we get complaints about
     // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
     // default localfs.  Presumption is that rootdir is fully-qualified before
@@ -317,8 +315,8 @@ public class MasterFileSystem {
       }
     }
 
-    if (this.metrics != null) {
-      this.metrics.addSplit(splitTime, splitLogSize);
+    if (this.metricsMaster != null) {
+      this.metricsMaster.addSplit(splitTime, splitLogSize);
     }
   }
 

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,92 @@
+/**
+ * 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.master;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.master.MetricsMasterSource;
+import org.apache.hadoop.hbase.master.MetricsMasterSourceFactory;
+import org.apache.hadoop.hbase.master.MetricsMasterWrapper;
+
+/**
+ * This class is for maintaining the various master statistics
+ * and publishing them through the metrics interfaces.
+ * <p>
+ * This class has a number of metrics variables that are publicly accessible;
+ * these variables (objects) have methods to update their values.
+ */
+@InterfaceStability.Evolving
+@InterfaceAudience.Private
+public class MetricsMaster {
+  private final Log LOG = LogFactory.getLog(this.getClass());
+  private MetricsMasterSource masterSource;
+
+  public MetricsMaster(MetricsMasterWrapper masterWrapper) {
+    masterSource = CompatibilitySingletonFactory.getInstance(MetricsMasterSourceFactory.class).create(masterWrapper);
+  }
+
+  // for unit-test usage
+  public MetricsMasterSource getMetricsSource() {
+    return masterSource;
+  }
+
+  /**
+   * Record a single instance of a split
+   * @param time time that the split took
+   * @param size length of original HLogs that were split
+   */
+  public synchronized void addSplit(long time, long size) {
+    masterSource.updateSplitTime(time);
+    masterSource.updateSplitSize(size);
+  }
+
+  /**
+   * @param inc How much to add to requests.
+   */
+  public void incrementRequests(final int inc) {
+    masterSource.incRequests(inc);
+
+  }
+
+  /**
+   * set new value for number of regions in transition.
+   * @param ritCount
+   */
+  public void updateRITCount(int ritCount) {
+    masterSource.setRIT(ritCount);
+  }
+
+  /**
+   * update RIT count that are in this state for more than the threshold
+   * as defined by the property rit.metrics.threshold.time.
+   * @param ritCountOverThreshold
+   */
+  public void updateRITCountOverThreshold(int ritCountOverThreshold) {
+    masterSource.setRITCountOverThreshold(ritCountOverThreshold);
+  }
+  /**
+   * update the timestamp for oldest region in transition metrics.
+   * @param timestamp
+   */
+  public void updateRITOldestAge(long timestamp) {
+    masterSource.setRITOldestAge(timestamp);
+  }
+}

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java?rev=1406396&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java (added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterWrapperImpl.java Tue Nov  6 23:22:01 2012
@@ -0,0 +1,101 @@
+/**
+ * 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.master;
+
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MetricsMasterWrapper;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+
+/**
+ * Impl for exposing HMaster Information through JMX
+ */
+public class MetricsMasterWrapperImpl implements MetricsMasterWrapper {
+
+  private final HMaster master;
+
+  public MetricsMasterWrapperImpl(final HMaster master) {
+    this.master = master;
+  }
+
+  @Override
+  public double getAverageLoad() {
+    return master.getAverageLoad();
+  }
+
+  @Override
+  public String getClusterId() {
+    return master.getClusterId();
+  }
+
+  @Override
+  public String getZookeeperQuorum() {
+    ZooKeeperWatcher zk = master.getZooKeeperWatcher();
+    if (zk == null) {
+      return "";
+    }
+    return zk.getQuorum();
+  }
+
+  @Override
+  public String[] getCoprocessors() {
+    return master.getCoprocessors();
+  }
+
+  @Override
+  public long getStartTime() {
+    return master.getMasterStartTime();
+  }
+
+  @Override
+  public long getActiveTime() {
+    return master.getMasterActiveTime();
+  }
+
+  @Override
+  public int getRegionServers() {
+    ServerManager serverManager = this.master.getServerManager();
+    if (serverManager == null) {
+      return 0;
+    }
+    return serverManager.getOnlineServers().size();
+  }
+
+  @Override
+  public int getDeadRegionServers() {
+    ServerManager serverManager = this.master.getServerManager();
+    if (serverManager == null) {
+      return 0;
+    }
+    return serverManager.getDeadServers().size();
+  }
+
+  @Override
+  public String getServerName() {
+    ServerName serverName = master.getServerName();
+    if (serverName == null) {
+      return "";
+    }
+    return serverName.getServerName();
+  }
+
+  @Override
+  public boolean getIsActiveMaster() {
+    return master.isActiveMaster();
+  }
+}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1406396&r1=1406395&r2=1406396&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Tue Nov  6 23:22:01 2012
@@ -81,7 +81,6 @@ import org.apache.hadoop.hbase.HConstant
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HServerInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -116,11 +115,7 @@ import org.apache.hadoop.hbase.ipc.HBase
 import org.apache.hadoop.hbase.ipc.RpcCallContext;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
-import org.apache.hadoop.hbase.regionserver.metrics.OperationMetrics;
-import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage;
-import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
@@ -235,16 +230,21 @@ public class HRegion implements HeapSize
   // private int [] storeSize = null;
   // private byte [] name = null;
 
-  final AtomicLong memstoreSize = new AtomicLong(0);
+  public final AtomicLong memstoreSize = new AtomicLong(0);
 
   // Debug possible data loss due to WAL off
-  final AtomicLong numPutsWithoutWAL = new AtomicLong(0);
-  final AtomicLong dataInMemoryWithoutWAL = new AtomicLong(0);
+  final Counter numPutsWithoutWAL = new Counter();
+  final Counter dataInMemoryWithoutWAL = new Counter();
 
+  // Debug why CAS operations are taking a while.
   final Counter checkAndMutateChecksPassed = new Counter();
   final Counter checkAndMutateChecksFailed = new Counter();
+
+  //Number of requests
   final Counter readRequestsCount = new Counter();
   final Counter writeRequestsCount = new Counter();
+
+  //How long operations were blocked by a memstore over highwater.
   final Counter updatesBlockedMs = new Counter();
 
   /**
@@ -362,7 +362,8 @@ public class HRegion implements HeapSize
   public final static String REGIONINFO_FILE = ".regioninfo";
   private HTableDescriptor htableDescriptor = null;
   private RegionSplitPolicy splitPolicy;
-  private final OperationMetrics opMetrics;
+
+  private final MetricsRegion metricsRegion;
 
   /**
    * Should only be used for testing purposes
@@ -386,7 +387,7 @@ public class HRegion implements HeapSize
     this.coprocessorHost = null;
     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
 
-    this.opMetrics = new OperationMetrics();
+    this.metricsRegion = new MetricsRegion(new MetricsRegionWrapperImpl(this));
   }
 
   /**
@@ -449,7 +450,7 @@ public class HRegion implements HeapSize
     this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
     this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
 
-    this.opMetrics = new OperationMetrics(conf, this.regionInfo);
+    this.metricsRegion = new MetricsRegion(new MetricsRegionWrapperImpl(this));
 
     /*
      * timestamp.slop provides a server-side constraint on the timestamp. This
@@ -839,21 +840,20 @@ public class HRegion implements HeapSize
     return this.rsServices;
   }
 
-  /** @return requestsCount for this region */
-  public long getRequestsCount() {
-    return this.readRequestsCount.get() + this.writeRequestsCount.get();
-  }
-
   /** @return readRequestsCount for this region */
-  public long getReadRequestsCount() {
+  long getReadRequestsCount() {
     return this.readRequestsCount.get();
   }
 
   /** @return writeRequestsCount for this region */
-  public long getWriteRequestsCount() {
+  long getWriteRequestsCount() {
     return this.writeRequestsCount.get();
   }
 
+  MetricsRegion getMetrics() {
+    return metricsRegion;
+  }
+
   /** @return true if region is closed */
   public boolean isClosed() {
     return this.closed.get();
@@ -1023,7 +1023,7 @@ public class HRegion implements HeapSize
         status.setStatus("Running coprocessor post-close hooks");
         this.coprocessorHost.postClose(abort);
       }
-      this.opMetrics.closeMetrics();
+      this.metricsRegion.close();
       status.markComplete("Closed");
       LOG.info("Closed " + this);
       return result;
@@ -1723,7 +1723,6 @@ public class HRegion implements HeapSize
   protected RegionScanner getScanner(Scan scan,
       List<KeyValueScanner> additionalScanners) throws IOException {
     startRegionOperation();
-    this.readRequestsCount.increment();
     try {
       // Verify families are all valid
       prepareScanner(scan);
@@ -2322,26 +2321,20 @@ public class HRegion implements HeapSize
         }
       }
 
-      // do after lock
-      final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis() - startTimeMs;
-
       // See if the column families were consistent through the whole thing.
       // if they were then keep them. If they were not then pass a null.
       // null will be treated as unknown.
       // Total time taken might be involving Puts and Deletes.
       // Split the time for puts and deletes based on the total number of Puts and Deletes.
-      long timeTakenForPuts = 0;
+
       if (noOfPuts > 0) {
         // There were some Puts in the batch.
         double noOfMutations = noOfPuts + noOfDeletes;
-        timeTakenForPuts = (long) (netTimeMs * (noOfPuts / noOfMutations));
-        final Set<byte[]> keptCfs = putsCfSetConsistent ? putsCfSet : null;
-        this.opMetrics.updateMultiPutMetrics(keptCfs, timeTakenForPuts);
+        this.metricsRegion.updatePut();
       }
       if (noOfDeletes > 0) {
         // There were some Deletes in the batch.
-        final Set<byte[]> keptCfs = deletesCfSetConsistent ? deletesCfSet : null;
-        this.opMetrics.updateMultiDeleteMetrics(keptCfs, netTimeMs - timeTakenForPuts);
+        this.metricsRegion.updateDelete();
       }
       if (!success) {
         for (int i = firstIndex; i < lastIndexExclusive; i++) {
@@ -3179,7 +3172,7 @@ public class HRegion implements HeapSize
 
   /**
    * See if row is currently locked.
-   * @param lockid
+   * @param lockId
    * @return boolean
    */
   boolean isRowLocked(final Integer lockId) {
@@ -4248,7 +4241,6 @@ public class HRegion implements HeapSize
    */
   private List<KeyValue> get(Get get, boolean withCoprocessor)
   throws IOException {
-    long now = EnvironmentEdgeManager.currentTimeMillis();
 
     List<KeyValue> results = new ArrayList<KeyValue>();
 
@@ -4264,7 +4256,7 @@ public class HRegion implements HeapSize
     RegionScanner scanner = null;
     try {
       scanner = getScanner(scan);
-      scanner.next(results, SchemaMetrics.METRIC_GETSIZE);
+      scanner.next(results);
     } finally {
       if (scanner != null)
         scanner.close();
@@ -4276,8 +4268,8 @@ public class HRegion implements HeapSize
     }
 
     // do after lock
-    final long after = EnvironmentEdgeManager.currentTimeMillis();
-    this.opMetrics.updateGetMetrics(get.familySet(), after - now);
+
+    this.metricsRegion.updateGet();
 
     return results;
   }
@@ -4324,9 +4316,6 @@ public class HRegion implements HeapSize
   public void processRowsWithLocks(RowProcessor<?> processor, long timeout)
       throws IOException {
 
-    final long startNanoTime = System.nanoTime();
-    String metricsName = "rowprocessor." + processor.getName();
-
     for (byte[] row : processor.getRowsToLock()) {
       checkRow(row, "processRowsWithLocks");
     }
@@ -4349,20 +4338,13 @@ public class HRegion implements HeapSize
             processor, now, this, null, null, timeout);
         processor.postProcess(this, walEdit);
       } catch (IOException e) {
-        long endNanoTime = System.nanoTime();
-        RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".error.nano",
-                                      endNanoTime - startNanoTime);
         throw e;
       } finally {
         closeRegionOperation();
       }
-      final long endNanoTime = System.nanoTime();
-      RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".nano",
-                                    endNanoTime - startNanoTime);
       return;
     }
 
-    long lockedNanoTime, processDoneNanoTime, unlockedNanoTime = 0;
     MultiVersionConsistencyControl.WriteEntry writeEntry = null;
     boolean locked = false;
     boolean walSyncSuccessful = false;
@@ -4385,7 +4367,6 @@ public class HRegion implements HeapSize
       // 3. Region lock
       this.updatesLock.readLock().lock();
       locked = true;
-      lockedNanoTime = System.nanoTime();
 
       long now = EnvironmentEdgeManager.currentTimeMillis();
       try {
@@ -4393,7 +4374,6 @@ public class HRegion implements HeapSize
         //    waledits
         doProcessRowWithTimeout(
             processor, now, this, mutations, walEdit, timeout);
-        processDoneNanoTime = System.nanoTime();
 
         if (!mutations.isEmpty()) {
           // 5. Get a mvcc write number
@@ -4418,7 +4398,6 @@ public class HRegion implements HeapSize
             this.updatesLock.readLock().unlock();
             locked = false;
           }
-          unlockedNanoTime = System.nanoTime();
 
           // 9. Release row lock(s)
           if (acquiredLocks != null) {
@@ -4456,17 +4435,13 @@ public class HRegion implements HeapSize
             releaseRowLock(lid);
           }
         }
-        unlockedNanoTime = unlockedNanoTime == 0 ?
-            System.nanoTime() : unlockedNanoTime;
+
       }
 
       // 12. Run post-process hook
       processor.postProcess(this, walEdit);
 
     } catch (IOException e) {
-      long endNanoTime = System.nanoTime();
-      RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".error.nano",
-                                    endNanoTime - startNanoTime);
       throw e;
     } finally {
       closeRegionOperation();
@@ -4475,22 +4450,6 @@ public class HRegion implements HeapSize
         requestFlush();
       }
     }
-    // Populate all metrics
-    long endNanoTime = System.nanoTime();
-    RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".nano",
-                                  endNanoTime - startNanoTime);
-
-    RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".acquirelock.nano",
-                                  lockedNanoTime - startNanoTime);
-
-    RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".process.nano",
-                                  processDoneNanoTime - lockedNanoTime);
-
-    RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".occupylock.nano",
-                                  unlockedNanoTime - lockedNanoTime);
-
-    RegionMetricsStorage.incrTimeVaryingMetric(metricsName + ".sync.nano",
-                                  endNanoTime - unlockedNanoTime);
   }
 
   private void doProcessRowWithTimeout(final RowProcessor<?> processor,
@@ -4567,7 +4526,7 @@ public class HRegion implements HeapSize
     WALEdit walEdits = null;
     List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
-    long before = EnvironmentEdgeManager.currentTimeMillis();
+
     long size = 0;
     long txid = 0;
 
@@ -4684,8 +4643,7 @@ public class HRegion implements HeapSize
       closeRegionOperation();
     }
 
-    long after = EnvironmentEdgeManager.currentTimeMillis();
-    this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
+    this.metricsRegion.updateAppend();
 
 
     if (flush) {
@@ -4720,7 +4678,7 @@ public class HRegion implements HeapSize
     WALEdit walEdits = null;
     List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
     Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
-    long before = EnvironmentEdgeManager.currentTimeMillis();
+
     long size = 0;
     long txid = 0;
 
@@ -4810,8 +4768,7 @@ public class HRegion implements HeapSize
       }
     } finally {
       closeRegionOperation();
-      long after = EnvironmentEdgeManager.currentTimeMillis();
-      this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
+      this.metricsRegion.updateIncrement();
     }
 
     if (flush) {
@@ -5284,7 +5241,8 @@ public class HRegion implements HeapSize
    * These information are exposed by the region server metrics.
    */
   private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
-    if (numPutsWithoutWAL.getAndIncrement() == 0) {
+    numPutsWithoutWAL.increment();
+    if (numPutsWithoutWAL.get() <= 1) {
       LOG.info("writing data to region " + this +
                " with WAL disabled. Data may be lost in the event of a crash.");
     }
@@ -5296,7 +5254,7 @@ public class HRegion implements HeapSize
       }
     }
 
-    dataInMemoryWithoutWAL.addAndGet(putSize);
+    dataInMemoryWithoutWAL.add(putSize);
   }
 
   /**