You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2017/08/23 14:54:05 UTC

hbase git commit: HBASE-18532 Improve cache related stats rendered on RS UI

Repository: hbase
Updated Branches:
  refs/heads/master dcd3e9abf -> 04f114b85


HBASE-18532 Improve cache related stats rendered on RS UI

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/master
Commit: 04f114b85c0a44351556ad0e73f7e65f8e3051a4
Parents: dcd3e9a
Author: Biju Nair <gs...@gmail.com>
Authored: Fri Aug 18 17:14:54 2017 -0400
Committer: tedyu <yu...@gmail.com>
Committed: Wed Aug 23 07:53:55 2017 -0700

----------------------------------------------------------------------
 .../hbase/io/hfile/MemcachedBlockCache.java     | 10 +++++++
 .../tmpl/regionserver/BlockCacheTmpl.jamon      |  8 +++---
 .../hadoop/hbase/io/hfile/BlockCache.java       | 12 ++++++++
 .../hbase/io/hfile/CombinedBlockCache.java      | 10 +++++++
 .../hadoop/hbase/io/hfile/LruBlockCache.java    | 30 +++++++++++++++++++-
 .../hbase/io/hfile/bucket/BucketCache.java      | 10 +++++++
 .../regionserver/TestHeapMemoryManager.java     | 10 +++++++
 7 files changed, 85 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
index e741760..965bcbe 100644
--- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
+++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
@@ -209,11 +209,21 @@ public class MemcachedBlockCache implements BlockCache {
   }
 
   @Override
+  public long getCurrentDataSize() {
+    return 0;
+  }
+
+  @Override
   public long getBlockCount() {
     return 0;
   }
 
   @Override
+  public long getDataBlockCount() {
+    return 0;
+  }
+
+  @Override
   public Iterator<CachedBlock> iterator() {
     return new Iterator<CachedBlock>() {
       @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/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 daa5d76..3afd4f9 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
@@ -327,25 +327,25 @@ are combined counts. Request count is sum of hits and misses.</p>
 </%if>
     <tr>
         <td>Count</td>
-        <td><% String.format("%,d", cbsbf.getCount()) %></td>
+        <td><% String.format("%,d", bc.getBlockCount()) %></td>
         <td>Count of Blocks</td>
     </tr>
 <%if !bucketCache %>
     <tr>
         <td>Count</td>
-        <td><% String.format("%,d", cbsbf.getDataCount()) %></td>
+        <td><% String.format("%,d", bc.getDataBlockCount()) %></td>
         <td>Count of DATA Blocks</td>
     </tr>
 </%if>
     <tr>
         <td>Size</td>
-        <td><% TraditionalBinaryPrefix.long2String(cbsbf.getSize(), "B", 1) %></td>
+        <td><% TraditionalBinaryPrefix.long2String(bc.getCurrentSize(), "B", 1) %></td>
         <td>Size of Blocks</td>
     </tr>
 <%if !bucketCache %>
     <tr>
         <td>Size</td>
-        <td><% TraditionalBinaryPrefix.long2String(cbsbf.getDataSize(), "B", 1) %></td>
+        <td><% TraditionalBinaryPrefix.long2String(bc.getCurrentDataSize(), "B", 1) %></td>
         <td>Size of DATA Blocks</td>
     </tr>
 </%if> 

http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java
index cef7e02..3674033 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java
@@ -103,10 +103,22 @@ public interface BlockCache extends Iterable<CachedBlock> {
   long getCurrentSize();
 
   /**
+   * Returns the occupied size of data blocks, in bytes.
+   * @return occupied space in cache, in bytes
+   */
+  long getCurrentDataSize();
+
+  /**
    * Returns the number of blocks currently cached in the block cache.
    * @return number of blocks in the cache
    */
   long getBlockCount();
+ 
+ /**
+  * Returns the number of data blocks currently cached in the block cache.
+  * @return number of blocks in the cache
+  */
+ long getDataBlockCount();
 
   /**
    * @return Iterator over the blocks in the cache.

http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 3efd3fe..abccdfc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -114,6 +114,11 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
   }
 
   @Override
+  public long getCurrentDataSize() {
+    return lruCache.getCurrentDataSize() + l2Cache.getCurrentDataSize();
+  }
+
+  @Override
   public long getFreeSize() {
     return lruCache.getFreeSize() + l2Cache.getFreeSize();
   }
@@ -128,6 +133,11 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
     return lruCache.getBlockCount() + l2Cache.getBlockCount();
   }
 
+  @Override
+  public long getDataBlockCount() {
+    return lruCache.getDataBlockCount() + l2Cache.getDataBlockCount();
+  }
+
   public static class CombinedCacheStats extends CacheStats {
     private final CacheStats lruCacheStats;
     private final CacheStats bucketCacheStats;

http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
index dbb0d49..ad81c50 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruBlockCache.java
@@ -177,9 +177,15 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   /** Current size of cache */
   private final AtomicLong size;
 
+  /** Current size of data blocks */
+  private final AtomicLong dataBlockSize;
+
   /** Current number of cached elements */
   private final AtomicLong elements;
 
+  /** Current number of cached data block elements */
+  private final AtomicLong dataBlockElements;
+
   /** Cache access count (sequential ID) */
   private final AtomicLong count;
 
@@ -315,6 +321,8 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
     this.stats = new CacheStats(this.getClass().getSimpleName());
     this.count = new AtomicLong(0);
     this.elements = new AtomicLong(0);
+    this.dataBlockElements = new AtomicLong(0);
+    this.dataBlockSize = new AtomicLong(0);
     this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
     this.size = new AtomicLong(this.overhead);
     this.hardCapacityLimitFactor = hardLimitFactor;
@@ -400,6 +408,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
     long newSize = updateSizeMetrics(cb, false);
     map.put(cacheKey, cb);
     long val = elements.incrementAndGet();
+    if (buf.getBlockType().isData()) {
+       dataBlockElements.incrementAndGet();
+    }
     if (LOG.isTraceEnabled()) {
       long size = map.size();
       assertCounterSanity(size, val);
@@ -455,9 +466,13 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
    */
   private long updateSizeMetrics(LruCachedBlock cb, boolean evict) {
     long heapsize = cb.heapSize();
+    BlockType bt = cb.getBuffer().getBlockType();
     if (evict) {
       heapsize *= -1;
     }
+    if (bt != null && bt.isData()) {
+       dataBlockSize.addAndGet(heapsize);
+    }
     return size.addAndGet(heapsize);
   }
 
@@ -562,6 +577,9 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
       long size = map.size();
       assertCounterSanity(size, val);
     }
+    if (block.getBuffer().getBlockType().isData()) {
+       dataBlockElements.decrementAndGet();
+    }
     if (evictedByEvictionProcess) {
       // When the eviction of the block happened because of invalidation of HFiles, no need to
       // update the stats counter.
@@ -832,6 +850,11 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   }
 
   @Override
+  public long getCurrentDataSize() {
+    return this.dataBlockSize.get();
+  }
+
+  @Override
   public long getFreeSize() {
     return getMaxSize() - getCurrentSize();
   }
@@ -846,6 +869,11 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
     return this.elements.get();
   }
 
+  @Override
+  public long getDataBlockCount() {
+    return this.dataBlockElements.get();
+  }
+
   EvictionThread getEvictionThread() {
     return this.evictionThread;
   }
@@ -959,7 +987,7 @@ public class LruBlockCache implements ResizableBlockCache, HeapSize {
   }
 
   public final static long CACHE_FIXED_OVERHEAD = ClassSize.align(
-      (4 * Bytes.SIZEOF_LONG) + (9 * ClassSize.REFERENCE) +
+      (4 * Bytes.SIZEOF_LONG) + (11 * ClassSize.REFERENCE) +
       (6 * Bytes.SIZEOF_FLOAT) + (2 * Bytes.SIZEOF_BOOLEAN)
       + ClassSize.OBJECT);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 79b1f4d..939d53a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -1190,6 +1190,11 @@ public class BucketCache implements BlockCache, HeapSize {
   }
 
   @Override
+  public long getCurrentDataSize() {
+    return size();
+  }
+
+  @Override
   public long getFreeSize() {
     return this.bucketAllocator.getFreeSize();
   }
@@ -1200,6 +1205,11 @@ public class BucketCache implements BlockCache, HeapSize {
   }
 
   @Override
+  public long getDataBlockCount() {
+    return getBlockCount();
+  }
+
+  @Override
   public long getCurrentSize() {
     return this.bucketAllocator.getUsedSize();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/04f114b8/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
index e1a8d5f..0840ac5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java
@@ -748,11 +748,21 @@ public class TestHeapMemoryManager {
     }
 
     @Override
+    public long getCurrentDataSize() {
+      return 0;
+    }
+
+    @Override
     public long getBlockCount() {
       return 0;
     }
 
     @Override
+    public long getDataBlockCount() {
+      return 0;
+    }
+
+    @Override
     public void setMaxSize(long size) {
       this.maxSize = size;
     }