HBASE-26982 Add index and bloom filter statistics of LruBlockCache on… (#4376)
Signed-off-by: Andrew Purtell <apurtell@apache.org> Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
9215066e2b
commit
8ec02c025e
|
@ -215,4 +215,13 @@ public enum BlockType {
|
|||
return this == DATA || this == ENCODED_DATA;
|
||||
}
|
||||
|
||||
/** Returns whether this block category is index */
|
||||
public final boolean isIndex() {
|
||||
return this.getCategory() == BlockCategory.INDEX;
|
||||
}
|
||||
|
||||
/** Returns whether this block category is bloom filter */
|
||||
public final boolean isBloom() {
|
||||
return this.getCategory() == BlockCategory.BLOOM;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ org.apache.hadoop.hbase.io.hfile.CachedBlock;
|
|||
org.apache.hadoop.conf.Configuration;
|
||||
org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
org.apache.hadoop.hbase.io.hfile.LruBlockCache;
|
||||
org.apache.hadoop.hbase.io.hfile.bucket.BucketCacheStats;
|
||||
org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
|
||||
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
|
||||
|
@ -284,6 +285,8 @@ are combined counts. Request count is sum of hits and misses.</p>
|
|||
String bcName = bc.getClass().getSimpleName();
|
||||
int maxCachedBlocksByFile = BlockCacheUtil.getMaxCachedBlocksByFile(config);
|
||||
|
||||
boolean lru = bc instanceof LruBlockCache;
|
||||
|
||||
boolean bucketCache = bc.getClass().getSimpleName().equals("BucketCache");
|
||||
BucketCacheStats bucketCacheStats = null;
|
||||
BucketAllocator bucketAllocator = null;
|
||||
|
@ -328,7 +331,19 @@ are combined counts. Request count is sum of hits and misses.</p>
|
|||
<td>Count of DATA Blocks</td>
|
||||
</tr>
|
||||
</%if>
|
||||
<%if lru %>
|
||||
<tr>
|
||||
<td>Index Block Count</td>
|
||||
<td><% String.format("%,d", ((LruBlockCache)bc).getIndexBlockCount()) %></td>
|
||||
<td>Count of INDEX Blocks</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Bloom Block Count</td>
|
||||
<td><% String.format("%,d", ((LruBlockCache)bc).getBloomBlockCount()) %></td>
|
||||
<td>Count of BLOOM Blocks</td>
|
||||
</tr>
|
||||
</%if>
|
||||
<tr>
|
||||
<td>Size of Blocks</td>
|
||||
<td><% TraditionalBinaryPrefix.long2String(bc.getCurrentSize(), "B", 1) %></td>
|
||||
<td>Size of Blocks</td>
|
||||
|
@ -340,7 +355,19 @@ are combined counts. Request count is sum of hits and misses.</p>
|
|||
<td>Size of DATA Blocks</td>
|
||||
</tr>
|
||||
</%if>
|
||||
<& evictions_tmpl; bc = bc; &>
|
||||
<%if lru %>
|
||||
<tr>
|
||||
<td>Size of Index Blocks</td>
|
||||
<td><% TraditionalBinaryPrefix.long2String(((LruBlockCache)bc).getCurrentIndexSize(), "B", 1) %></td>
|
||||
<td>Size of INDEX Blocks</td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>Size of Bloom Blocks</td>
|
||||
<td><% TraditionalBinaryPrefix.long2String(((LruBlockCache)bc).getCurrentBloomSize(), "B", 1) %></td>
|
||||
<td>Size of BLOOM Blocks</td>
|
||||
</tr>
|
||||
</%if>
|
||||
<& evictions_tmpl; bc = bc; &>
|
||||
<& hits_tmpl; bc = bc; &>
|
||||
|
||||
<%if bucketCache %>
|
||||
|
|
|
@ -174,13 +174,25 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
private final AtomicLong size;
|
||||
|
||||
/** Current size of data blocks */
|
||||
private final LongAdder dataBlockSize;
|
||||
private final LongAdder dataBlockSize = new LongAdder();
|
||||
|
||||
/** Current size of index blocks */
|
||||
private final LongAdder indexBlockSize = new LongAdder();
|
||||
|
||||
/** Current size of bloom blocks */
|
||||
private final LongAdder bloomBlockSize = new LongAdder();
|
||||
|
||||
/** Current number of cached elements */
|
||||
private final AtomicLong elements;
|
||||
|
||||
/** Current number of cached data block elements */
|
||||
private final LongAdder dataBlockElements;
|
||||
private final LongAdder dataBlockElements = new LongAdder();
|
||||
|
||||
/** Current number of cached index block elements */
|
||||
private final LongAdder indexBlockElements = new LongAdder();
|
||||
|
||||
/** Current number of cached bloom block elements */
|
||||
private final LongAdder bloomBlockElements = new LongAdder();
|
||||
|
||||
/** Cache access count (sequential ID) */
|
||||
private final AtomicLong count;
|
||||
|
@ -307,8 +319,6 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
this.stats = new CacheStats(this.getClass().getSimpleName());
|
||||
this.count = new AtomicLong(0);
|
||||
this.elements = new AtomicLong(0);
|
||||
this.dataBlockElements = new LongAdder();
|
||||
this.dataBlockSize = new LongAdder();
|
||||
this.overhead = calculateOverhead(maxSize, blockSize, mapConcurrencyLevel);
|
||||
this.size = new AtomicLong(this.overhead);
|
||||
this.hardCapacityLimitFactor = hardLimitFactor;
|
||||
|
@ -417,7 +427,11 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
long newSize = updateSizeMetrics(cb, false);
|
||||
map.put(cacheKey, cb);
|
||||
long val = elements.incrementAndGet();
|
||||
if (buf.getBlockType().isData()) {
|
||||
if (buf.getBlockType().isBloom()) {
|
||||
bloomBlockElements.increment();
|
||||
} else if (buf.getBlockType().isIndex()) {
|
||||
indexBlockElements.increment();
|
||||
} else if (buf.getBlockType().isData()) {
|
||||
dataBlockElements.increment();
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
|
@ -473,8 +487,14 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
if (evict) {
|
||||
heapsize *= -1;
|
||||
}
|
||||
if (bt != null && bt.isData()) {
|
||||
dataBlockSize.add(heapsize);
|
||||
if (bt != null) {
|
||||
if (bt.isBloom()) {
|
||||
bloomBlockSize.add(heapsize);
|
||||
} else if (bt.isIndex()) {
|
||||
indexBlockSize.add(heapsize);
|
||||
} else if (bt.isData()) {
|
||||
dataBlockSize.add(heapsize);
|
||||
}
|
||||
}
|
||||
return size.addAndGet(heapsize);
|
||||
}
|
||||
|
@ -582,7 +602,12 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
long size = map.size();
|
||||
assertCounterSanity(size, val);
|
||||
}
|
||||
if (block.getBuffer().getBlockType().isData()) {
|
||||
BlockType bt = block.getBuffer().getBlockType();
|
||||
if (bt.isBloom()) {
|
||||
bloomBlockElements.decrement();
|
||||
} else if (bt.isIndex()) {
|
||||
indexBlockElements.decrement();
|
||||
} else if (bt.isData()) {
|
||||
dataBlockElements.decrement();
|
||||
}
|
||||
if (evictedByEvictionProcess) {
|
||||
|
@ -851,6 +876,14 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
return this.dataBlockSize.sum();
|
||||
}
|
||||
|
||||
public long getCurrentIndexSize() {
|
||||
return this.indexBlockSize.sum();
|
||||
}
|
||||
|
||||
public long getCurrentBloomSize() {
|
||||
return this.bloomBlockSize.sum();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFreeSize() {
|
||||
return getMaxSize() - getCurrentSize();
|
||||
|
@ -871,6 +904,14 @@ public class LruBlockCache implements FirstLevelBlockCache {
|
|||
return this.dataBlockElements.sum();
|
||||
}
|
||||
|
||||
public long getIndexBlockCount() {
|
||||
return this.indexBlockElements.sum();
|
||||
}
|
||||
|
||||
public long getBloomBlockCount() {
|
||||
return this.bloomBlockElements.sum();
|
||||
}
|
||||
|
||||
EvictionThread getEvictionThread() {
|
||||
return this.evictionThread;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue