hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject svn commit: r1164349 - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/io/hfile/slab/ src/test/java/org/apache/hadoop/hbase/io/hfile/slab/
Date Fri, 02 Sep 2011 04:20:04 GMT
Author: tedyu
Date: Fri Sep  2 04:20:04 2011
New Revision: 1164349

URL: http://svn.apache.org/viewvc?rev=1164349&view=rev
Log:
HBASE-4310  SlabCache metrics bugfix (Li Pi)

Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java
    hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/slab/TestSlabCache.java

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1164349&r1=1164348&r2=1164349&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Fri Sep  2 04:20:04 2011
@@ -233,6 +233,7 @@ Release 0.91.0 - Unreleased
    HBASE-4315  RPC logging too verbose (todd)
    HBASE-4273  java.lang.NullPointerException when a table is being disabled and
                HMaster restarts (Ming Ma)
+   HBASE-4310  SlabCache metrics bugfix (Li Pi)
 
   IMPROVEMENTS
    HBASE-3290  Max Compaction Size (Nicolas Spiegelberg via Stack)  

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java?rev=1164349&r1=1164348&r2=1164349&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SingleSizeCache.java Fri
Sep  2 04:20:04 2011
@@ -61,6 +61,7 @@ public class SingleSizeCache implements 
   private final CacheStats stats;
   private final SlabItemEvictionWatcher evictionWatcher;
   private AtomicLong size;
+  private AtomicLong timeSinceLastAccess;
   public final static long CACHE_FIXED_OVERHEAD = ClassSize
       .align((2 * Bytes.SIZEOF_INT) + (5 * ClassSize.REFERENCE)
           + +ClassSize.OBJECT);
@@ -86,6 +87,7 @@ public class SingleSizeCache implements 
     this.stats = new CacheStats();
     this.evictionWatcher = master;
     this.size = new AtomicLong(CACHE_FIXED_OVERHEAD + backingStore.heapSize());
+    this.timeSinceLastAccess = new AtomicLong();
 
     // This evictionListener is called whenever the cache automatically evicts
     // something.
@@ -94,6 +96,8 @@ public class SingleSizeCache implements 
       public void onEviction(String key, CacheablePair value) {
         try {
           value.evictionLock.writeLock().lock();
+          timeSinceLastAccess.set(System.nanoTime()
+              - value.recentlyAccessed.get());
           backingStore.free(value.serializedData);
           stats.evict();
           /**
@@ -139,6 +143,7 @@ public class SingleSizeCache implements 
       throw new RuntimeException("already cached " + blockName);
     }
     toBeCached.serialize(storedBlock);
+    newEntry.recentlyAccessed.set(System.nanoTime());
     this.size.addAndGet(newEntry.heapSize());
   }
 
@@ -154,6 +159,7 @@ public class SingleSizeCache implements 
     // If lock cannot be obtained, that means we're undergoing eviction.
     if (contentBlock.evictionLock.readLock().tryLock()) {
       try {
+        contentBlock.recentlyAccessed.set(System.nanoTime());
         return contentBlock.deserializer
             .deserialize(contentBlock.serializedData);
       } catch (IOException e) {
@@ -193,11 +199,14 @@ public class SingleSizeCache implements 
 
   public void logStats() {
 
+    long milliseconds = (long)this.timeSinceLastAccess.get() / 1000000;
+
     LOG.info("For Slab of size " + this.blockSize + ": "
         + this.getOccupiedSize() / this.blockSize
         + " occupied, out of a capacity of " + this.numBlocks
         + " blocks. HeapSize is "
-        + StringUtils.humanReadableInt(this.heapSize()) + " bytes.");
+        + StringUtils.humanReadableInt(this.heapSize()) + " bytes." + ", "
+        + "churnTime=" + StringUtils.formatTime(milliseconds));
 
     LOG.debug("Slab Stats: " + "accesses="
         + stats.getRequestCount()
@@ -292,9 +301,11 @@ public class SingleSizeCache implements 
     final CacheableDeserializer<Cacheable> deserializer;
     final ByteBuffer serializedData;
     final ReentrantReadWriteLock evictionLock;
+    AtomicLong recentlyAccessed;
 
     private CacheablePair(CacheableDeserializer<Cacheable> deserializer,
         ByteBuffer serializedData) {
+      this.recentlyAccessed = new AtomicLong();
       this.deserializer = deserializer;
       this.serializedData = serializedData;
       evictionLock = new ReentrantReadWriteLock();

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java?rev=1164349&r1=1164348&r2=1164349&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java Fri Sep
 2 04:20:04 2011
@@ -175,6 +175,8 @@ public class SlabCache implements SlabIt
   }
 
   private void addSlab(int blockSize, int numBlocks) {
+    LOG.info("Creating a slab of blockSize " + blockSize + " with " + numBlocks
+        + " blocks.");
     sizer.put(blockSize, new SingleSizeCache(blockSize, numBlocks, this));
   }
 
@@ -332,8 +334,8 @@ public class SlabCache implements SlabIt
   static class SlabStats {
     // the maximum size somebody will ever try to cache, then we multiply by 10
     // so we have finer grained stats.
-    private final int MULTIPLIER = 10;
-    private final int NUMDIVISIONS = (int) (Math.log(Integer.MAX_VALUE) * MULTIPLIER);
+    final int MULTIPLIER = 10;
+    final int NUMDIVISIONS = (int) (Math.log(Integer.MAX_VALUE) * MULTIPLIER);
     private final AtomicLong[] counts = new AtomicLong[NUMDIVISIONS];
 
     public SlabStats() {
@@ -351,24 +353,27 @@ public class SlabCache implements SlabIt
       return counts;
     }
 
+    double getUpperBound(int index) {
+      return Math.pow(Math.E, ((double) (index + 0.5) / (double) MULTIPLIER));
+    }
+
+    double getLowerBound(int index) {
+      return Math.pow(Math.E, ((double) (index - 0.5) / (double) MULTIPLIER));
+    }
+
     public void logStats(SlabCache slabCache) {
       for (SingleSizeCache s : slabCache.sizer.values()) {
         s.logStats();
       }
       AtomicLong[] fineGrainedStats = getUsage();
-      int multiplier = MULTIPLIER;
       SlabCache.LOG.info("Current heap size is: "
           + StringUtils.humanReadableInt(slabCache.heapSize()));
       for (int i = 0; i < fineGrainedStats.length; i++) {
-        double lowerbound = Math.pow(Math.E,
-            ((double) i / (double) multiplier) - 0.5);
-        double upperbound = Math.pow(Math.E,
-            ((double) i / (double) multiplier) + 0.5);
 
         if (fineGrainedStats[i].get() > 0) {
           SlabCache.LOG.info("From  "
-              + StringUtils.humanReadableInt((long) lowerbound) + "- "
-              + StringUtils.humanReadableInt((long) upperbound) + ": "
+              + StringUtils.humanReadableInt((long) getLowerBound(i)) + "- "
+              + StringUtils.humanReadableInt((long) getUpperBound(i)) + ": "
               + StringUtils.humanReadableInt(fineGrainedStats[i].get())
               + " requests");
 

Modified: hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/slab/TestSlabCache.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/slab/TestSlabCache.java?rev=1164349&r1=1164348&r2=1164349&view=diff
==============================================================================
--- hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/slab/TestSlabCache.java (original)
+++ hbase/trunk/src/test/java/org/apache/hadoop/hbase/io/hfile/slab/TestSlabCache.java Fri
Sep  2 04:20:04 2011
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.io.hfile
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
 import org.apache.hadoop.hbase.io.hfile.slab.SlabCache;
+import org.apache.hadoop.hbase.io.hfile.slab.SlabCache.SlabStats;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -80,4 +81,13 @@ public class TestSlabCache {
   public void testCacheMultiThreadedSingleKey() throws Exception {
     CacheTestUtils.hammerSingleKey(cache, BLOCK_SIZE, NUM_THREADS, NUM_QUERIES);
   }
+
+  @Test
+  /*Just checks if ranges overlap*/
+  public void testStatsArithmetic(){
+    SlabStats test = cache.requestStats;
+    for(int i = 0; i < test.NUMDIVISIONS; i++){
+      assert(test.getUpperBound(i) < test.getLowerBound(i + 1));
+    }
+  }
 }



Mime
View raw message