You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/02/27 00:09:36 UTC

[15/37] hbase git commit: HBASE-15222 Use less contended classes for metrics

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/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
index 32d4fae..aaf4359 100644
--- 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
@@ -18,8 +18,6 @@
 
 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;
 
@@ -28,31 +26,30 @@ import org.apache.hadoop.metrics2.MetricsInfo;
  */
 @InterfaceAudience.Private
 public class MutableTimeHistogram extends MutableRangeHistogram {
-  private final String rangeType = "TimeRangeCount";
-  private final long[] ranges =
+  private final static String RANGE_TYPE = "TimeRangeCount";
+  private final static 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);
+    this(name, description, RANGES[RANGES.length - 2]);
+  }
+
+  public MutableTimeHistogram(String name, String description, long expectedMax) {
+    super(name, description, expectedMax);
   }
 
   @Override
   public String getRangeType() {
-    return rangeType;
+    return RANGE_TYPE;
   }
 
   @Override
-  public long[] getRange() {
-    return ranges;
+  public long[] getRanges() {
+    return RANGES;
   }
 
-  @Override
-  public AtomicLongArray getRangeVals() {
-    return rangeVals;
-  } 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
index 7381fb9..2e374f7 100644
--- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
+++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/metrics/TestBaseSourceImpl.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.metrics;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.testclassification.MetricsTests;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableFastCounter;
 import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -72,9 +73,9 @@ public class TestBaseSourceImpl {
   @Test
   public void testIncCounters() throws Exception {
     bmsi.incCounters("testinccounter", 100);
-    assertEquals(100, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
+    assertEquals(100, ((MutableFastCounter) bmsi.metricsRegistry.get("testinccounter")).value());
     bmsi.incCounters("testinccounter", 100);
-    assertEquals(200, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
+    assertEquals(200, ((MutableFastCounter) bmsi.metricsRegistry.get("testinccounter")).value());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
index 6986f12..3dcd5e2 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
@@ -178,7 +178,6 @@ org.apache.hadoop.util.StringUtils;
   AgeSnapshot ageAtEvictionSnapshot = bc.getStats().getAgeAtEvictionSnapshot();
   // Only show if non-zero mean and stddev as is the case in combinedblockcache
   double mean = ageAtEvictionSnapshot.getMean();
-  double stddev = ageAtEvictionSnapshot.getStdDev();
 </%java>
     <tr>
         <td>Evicted</td>
@@ -197,13 +196,6 @@ org.apache.hadoop.util.StringUtils;
         <td>Mean age of Blocks at eviction time (seconds)</td>
     </tr>
 </%if>
-<%if stddev > 0 %>
-    <tr>
-        <td>StdDev</td>
-        <td><% String.format("%,d", (long)(ageAtEvictionSnapshot.getStdDev()/1000000)) %></td>
-        <td>Standard Deviation for age of Blocks at eviction time</td>
-    </tr>
-</%if>
 </%def>
 
 <%def hits_tmpl>

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
index c23cf75..fa55f6a 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
@@ -37,7 +37,6 @@ org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
 org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
 org.apache.hadoop.util.StringUtils;
-com.codahale.metrics.Snapshot;
 </%import>
 <%java>
   BlockCache bc = cacheConfig == null ? null : cacheConfig.getBlockCache();

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
index a55a863..de2aeca 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon
@@ -34,7 +34,6 @@ org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
 org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
 org.apache.hadoop.hbase.util.DirectMemoryUtils;
 org.apache.hadoop.util.StringUtils;
-com.codahale.metrics.Snapshot;
 java.lang.management.ManagementFactory;
 </%import>
 <div class="tabbable">

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
index 79acec0..4c1ad23 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/AgeSnapshot.java
@@ -17,56 +17,54 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.Snapshot;
-
 /**
  * Snapshot of block cache age in cache.
  * This object is preferred because we can control how it is serialized out when JSON'ing.
  */
 @JsonIgnoreProperties({"ageHistogram", "snapshot"})
 public class AgeSnapshot {
-  private final Snapshot snapshot;
 
-  AgeSnapshot(final Histogram ageHistogram) {
-    this.snapshot = ageHistogram.getSnapshot();
+  private final FastLongHistogram ageHistogram;
+  private final long[] quantiles;
+
+  AgeSnapshot(final FastLongHistogram ageHistogram) {
+    this.ageHistogram = ageHistogram;
+    this.quantiles = ageHistogram.getQuantiles(new double[]{0.75, 0.95, 0.98, 0.99, 0.999});
   }
 
   public double get75thPercentile() {
-    return snapshot.get75thPercentile();
+    return quantiles[0];
   }
 
   public double get95thPercentile() {
-    return snapshot.get95thPercentile();
+    return quantiles[1];
   }
 
   public double get98thPercentile() {
-    return snapshot.get98thPercentile();
+    return quantiles[2];
   }
 
-  public double get999thPercentile() {
-    return snapshot.get999thPercentile();
+  public double get99thPercentile() {
+    return quantiles[3];
   }
 
-  public double get99thPercentile() {
-    return snapshot.get99thPercentile();
+  public double get999thPercentile() {
+    return quantiles[4];
   }
 
+
   public double getMean() {
-    return this.snapshot.getMean();
+    return this.ageHistogram.getMean();
   }
 
   public double getMax() {
-    return snapshot.getMax();
+    return this.ageHistogram.getMax();
   }
 
   public double getMin() {
-    return snapshot.getMin();
-  }
-
-  public double getStdDev() {
-    return snapshot.getStdDev();
+    return this.ageHistogram.getMin();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
index d81871f..977284b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheUtil.java
@@ -25,17 +25,13 @@ import java.util.concurrent.ConcurrentSkipListSet;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 import org.codehaus.jackson.JsonGenerationException;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.JsonMappingException;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
 
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.Snapshot;
-
-import static com.codahale.metrics.MetricRegistry.name;
 
 /**
  * Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
@@ -44,11 +40,6 @@ import static com.codahale.metrics.MetricRegistry.name;
 @InterfaceAudience.Private
 public class BlockCacheUtil {
   /**
-   * Needed making histograms.
-   */
-  private static final MetricRegistry METRICS = new MetricRegistry();
-
-  /**
    * Needed generating JSON.
    */
   private static final ObjectMapper MAPPER = new ObjectMapper();
@@ -206,7 +197,7 @@ public class BlockCacheUtil {
      */
     private NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
       new ConcurrentSkipListMap<String, NavigableSet<CachedBlock>>();
-    Histogram age = METRICS.histogram(name(CachedBlocksByFile.class, "age"));
+    FastLongHistogram hist = new FastLongHistogram();
 
     /**
      * @param cb
@@ -228,7 +219,7 @@ public class BlockCacheUtil {
         this.dataSize += cb.getSize();
       }
       long age = this.now - cb.getCachedTime();
-      this.age.update(age);
+      this.hist.add(age, 1);
       return false;
     }
 
@@ -271,18 +262,22 @@ public class BlockCacheUtil {
     }
 
     public AgeSnapshot getAgeInCacheSnapshot() {
-      return new AgeSnapshot(this.age);
+      return new AgeSnapshot(this.hist);
     }
 
     @Override
     public String toString() {
-      Snapshot snapshot = age.getSnapshot();
+      AgeSnapshot snapshot = getAgeInCacheSnapshot();
       return "count=" + count + ", dataBlockCount=" + dataBlockCount + ", size=" + size +
           ", dataSize=" + getDataSize() +
-          ", mean age=" + snapshot.getMean() + ", stddev age=" + snapshot.getStdDev() +
-          ", min age=" + snapshot.getMin() + ", max age=" + snapshot.getMax() +
-          ", 95th percentile age=" + snapshot.get95thPercentile() +
-          ", 99th percentile age=" + snapshot.get99thPercentile();
+          ", mean age=" + snapshot.getMean() +
+          ", min age=" + snapshot.getMin() +
+          ", max age=" + snapshot.getMax() +
+          ", 75th percentile age="   + snapshot.get75thPercentile() +
+          ", 95th percentile age="   + snapshot.get95thPercentile() +
+          ", 98th percentile age="   + snapshot.get98thPercentile() +
+          ", 99th percentile age="   + snapshot.get99thPercentile() +
+          ", 99.9th percentile age=" + snapshot.get99thPercentile();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
index 50e8bbb..c208388 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheStats.java
@@ -22,20 +22,15 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.codahale.metrics.Histogram;
-import com.codahale.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.util.Counter;
+import org.apache.hadoop.hbase.util.FastLongHistogram;
 
-import static com.codahale.metrics.MetricRegistry.name;
 
 /**
  * Class that implements cache metrics.
  */
 @InterfaceAudience.Private
 public class CacheStats {
-  /**
-   * Needed making histograms.
-   */
-  private static final MetricRegistry METRICS = new MetricRegistry();
 
   /** Sliding window statistics. The number of metric periods to include in
    * sliding window hit ratio calculations.
@@ -43,10 +38,10 @@ public class CacheStats {
   static final int DEFAULT_WINDOW_PERIODS = 5;
 
   /** The number of getBlock requests that were cache hits */
-  private final AtomicLong hitCount = new AtomicLong(0);
+  private final Counter hitCount = new Counter();
 
   /** The number of getBlock requests that were cache hits from primary replica */
-  private final AtomicLong primaryHitCount = new AtomicLong(0);
+  private final Counter primaryHitCount = new Counter();
   
   /**
    * The number of getBlock requests that were cache hits, but only from
@@ -54,27 +49,27 @@ public class CacheStats {
    * attempt to read from the block cache even if they will not put new blocks
    * into the block cache.  See HBASE-2253 for more information.
    */
-  private final AtomicLong hitCachingCount = new AtomicLong(0);
+  private final Counter hitCachingCount = new Counter();
 
   /** The number of getBlock requests that were cache misses */
-  private final AtomicLong missCount = new AtomicLong(0);
+  private final Counter missCount = new Counter();
 
   /** The number of getBlock requests for primary replica that were cache misses */
-  private final AtomicLong primaryMissCount = new AtomicLong(0);
+  private final Counter primaryMissCount = new Counter();
   /**
    * The number of getBlock requests that were cache misses, but only from
    * requests that were set to use the block cache.
    */
-  private final AtomicLong missCachingCount = new AtomicLong(0);
+  private final Counter missCachingCount = new Counter();
 
   /** The number of times an eviction has occurred */
-  private final AtomicLong evictionCount = new AtomicLong(0);
+  private final Counter evictionCount = new Counter();
 
   /** The total number of blocks that have been evicted */
-  private final AtomicLong evictedBlockCount = new AtomicLong(0);
+  private final Counter evictedBlockCount = new Counter();
 
   /** The total number of blocks for primary replica that have been evicted */
-  private final AtomicLong primaryEvictedBlockCount = new AtomicLong(0);
+  private final Counter primaryEvictedBlockCount = new Counter();
 
   /** The total number of blocks that were not inserted. */
   private final AtomicLong failedInserts = new AtomicLong(0);
@@ -102,7 +97,7 @@ public class CacheStats {
   /**
    * Keep running age at eviction time
    */
-  private Histogram ageAtEviction;
+  private FastLongHistogram ageAtEviction;
   private long startTime = System.nanoTime();
 
   public CacheStats(final String name) {
@@ -115,7 +110,7 @@ public class CacheStats {
     this.hitCachingCounts = initializeZeros(numPeriodsInWindow);
     this.requestCounts = initializeZeros(numPeriodsInWindow);
     this.requestCachingCounts = initializeZeros(numPeriodsInWindow);
-    this.ageAtEviction = METRICS.histogram(name(CacheStats.class, name + ".ageAtEviction"));
+    this.ageAtEviction = new FastLongHistogram();
   }
 
   @Override
@@ -127,14 +122,13 @@ public class CacheStats {
       ", evictedBlockCount=" + getEvictedCount() +
       ", primaryMissCount=" + getPrimaryMissCount() +
       ", primaryHitCount=" + getPrimaryHitCount() +
-      ", evictedAgeMean=" + snapshot.getMean() +
-      ", evictedAgeStdDev=" + snapshot.getStdDev();
+      ", evictedAgeMean=" + snapshot.getMean();
   }
 
   public void miss(boolean caching, boolean primary) {
-    missCount.incrementAndGet();
-    if (primary) primaryMissCount.incrementAndGet();
-    if (caching) missCachingCount.incrementAndGet();
+    missCount.increment();
+    if (primary) primaryMissCount.increment();
+    if (caching) missCachingCount.increment();
   }
 
   public void hit(boolean caching) {
@@ -142,20 +136,20 @@ public class CacheStats {
   }
 
   public void hit(boolean caching, boolean primary) {
-    hitCount.incrementAndGet();
-    if (primary) primaryHitCount.incrementAndGet();
-    if (caching) hitCachingCount.incrementAndGet();
+    hitCount.increment();
+    if (primary) primaryHitCount.increment();
+    if (caching) hitCachingCount.increment();
   }
 
   public void evict() {
-    evictionCount.incrementAndGet();
+    evictionCount.increment();
   }
 
   public void evicted(final long t, boolean primary) {
-    if (t > this.startTime) this.ageAtEviction.update(t - this.startTime);
-    this.evictedBlockCount.incrementAndGet();
+    if (t > this.startTime) this.ageAtEviction.add(t - this.startTime,1);
+    this.evictedBlockCount.increment();
     if (primary) {
-      primaryEvictedBlockCount.incrementAndGet();
+      primaryEvictedBlockCount.increment();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
index 1e1835f..e9fa05c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java
@@ -34,7 +34,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.commons.logging.Log;
@@ -61,6 +60,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
 import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
 import org.apache.hadoop.hbase.util.BloomFilterWriter;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Counter;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.io.Writable;
 
@@ -179,17 +179,19 @@ public class HFile {
    */
   public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
   // For measuring number of checksum failures
-  static final AtomicLong checksumFailures = new AtomicLong();
+  static final Counter checksumFailures = new Counter();
 
   // for test purpose
-  public static final AtomicLong dataBlockReadCnt = new AtomicLong(0);
+  public static final Counter dataBlockReadCnt = new Counter();
 
   /**
    * Number of checksum verification failures. It also
    * clears the counter.
    */
   public static final long getChecksumFailuresCount() {
-    return checksumFailures.getAndSet(0);
+    long count = checksumFailures.get();
+    checksumFailures.set(0);
+    return count;
   }
 
   /** API required to write an {@link HFile} */

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index e7a1e5e..e2f524c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1522,7 +1522,7 @@ public class HFileBlock implements Cacheable {
           HFile.LOG.warn(msg);
           throw new IOException(msg); // cannot happen case here
         }
-        HFile.checksumFailures.incrementAndGet(); // update metrics
+        HFile.checksumFailures.increment(); // update metrics
 
         // If we have a checksum failure, we fall back into a mode where
         // the next few reads use HDFS level checksums. We aim to make the

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index b2f5ded..239c63d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -1497,7 +1497,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
             assert cachedBlock.isUnpacked() : "Packed block leak.";
             if (cachedBlock.getBlockType().isData()) {
               if (updateCacheMetrics) {
-                HFile.dataBlockReadCnt.incrementAndGet();
+                HFile.dataBlockReadCnt.increment();
               }
               // Validate encoding type for data blocks. We include encoding
               // type in the cache key, and we expect it to match on a cache hit.
@@ -1537,7 +1537,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         }
 
         if (updateCacheMetrics && hfileBlock.getBlockType().isData()) {
-          HFile.dataBlockReadCnt.incrementAndGet();
+          HFile.dataBlockReadCnt.increment();
         }
 
         return unpacked;

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
index 51e6268..ec1d4d9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCacheStats.java
@@ -18,10 +18,9 @@
  */
 package org.apache.hadoop.hbase.io.hfile.bucket;
 
-import java.util.concurrent.atomic.AtomicLong;
-
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
+import org.apache.hadoop.hbase.util.Counter;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
 /**
@@ -29,8 +28,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
  */
 @InterfaceAudience.Private
 public class BucketCacheStats extends CacheStats {
-  private final AtomicLong ioHitCount = new AtomicLong(0);
-  private final AtomicLong ioHitTime = new AtomicLong(0);
+  private final Counter ioHitCount = new Counter(0);
+  private final Counter ioHitTime = new Counter(0);
   private final static int nanoTime = 1000000;
   private long lastLogTime = EnvironmentEdgeManager.currentTime();
 
@@ -45,8 +44,8 @@ public class BucketCacheStats extends CacheStats {
   }
 
   public void ioHit(long time) {
-    ioHitCount.incrementAndGet();
-    ioHitTime.addAndGet(time);
+    ioHitCount.increment();
+    ioHitTime.add(time);
   }
 
   public long getIOHitsPerSecond() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/630a6582/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
index d752e17..924e7f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java
@@ -24,7 +24,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Cell;
@@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
+import org.apache.hadoop.hbase.util.Counter;
 
 /**
  * KeyValueScanner adaptor over the Reader.  It also provides hooks into
@@ -60,7 +60,7 @@ public class StoreFileScanner implements KeyValueScanner {
   // if have encountered the next row. Only used for reversed scan
   private boolean stopSkippingKVsIfNextRow = false;
 
-  private static AtomicLong seekCount;
+  private static Counter seekCount;
 
   private ScanQueryMatcher matcher;
 
@@ -164,7 +164,7 @@ public class StoreFileScanner implements KeyValueScanner {
   }
 
   public boolean seek(Cell key) throws IOException {
-    if (seekCount != null) seekCount.incrementAndGet();
+    if (seekCount != null) seekCount.increment();
 
     try {
       try {
@@ -191,7 +191,7 @@ public class StoreFileScanner implements KeyValueScanner {
   }
 
   public boolean reseek(Cell key) throws IOException {
-    if (seekCount != null) seekCount.incrementAndGet();
+    if (seekCount != null) seekCount.increment();
 
     try {
       try {
@@ -424,7 +424,7 @@ public class StoreFileScanner implements KeyValueScanner {
     return seekCount.get();
   }
   static final void instrument() {
-    seekCount = new AtomicLong();
+    seekCount = new Counter();
   }
 
   @Override
@@ -447,7 +447,7 @@ public class StoreFileScanner implements KeyValueScanner {
         Cell key = originalKey;
         do {
           Cell seekKey = CellUtil.createFirstOnRow(key);
-          if (seekCount != null) seekCount.incrementAndGet();
+          if (seekCount != null) seekCount.increment();
           if (!hfs.seekBefore(seekKey)) {
             this.cur = null;
             return false;
@@ -455,7 +455,7 @@ public class StoreFileScanner implements KeyValueScanner {
           Cell curCell = hfs.getCell();
           Cell firstKeyOfPreviousRow = CellUtil.createFirstOnRow(curCell);
 
-          if (seekCount != null) seekCount.incrementAndGet();
+          if (seekCount != null) seekCount.increment();
           if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
             this.cur = null;
             return false;