Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-26982 Add index and bloom filter statistics of LruBlockCache on… #4376

Merged
merged 4 commits into from
Aug 15, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -219,4 +219,17 @@ public final boolean isData() {
return this == DATA || this == ENCODED_DATA;
}

/**
* @return whether this block category is index
*/
public final boolean isIndex() {
return this.getCategory() == BlockCategory.INDEX;
}

/**
* @return whether this block category is bloom filter
*/
public final boolean isBloom() {
return this.getCategory() == BlockCategory.BLOOM;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,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;
Expand Down Expand Up @@ -285,6 +286,8 @@ are combined counts. Request count is sum of hits and misses.</p>
org.apache.hadoop.hbase.io.hfile.BlockCacheUtil.getLoadedCachedBlocksByFile(config, bc);
AgeSnapshot cbsbfSnapshot = cbsbf.getAgeInCacheSnapshot();

boolean lru = bc instanceof LruBlockCache;

boolean bucketCache = bc.getClass().getSimpleName().equals("BucketCache");
BucketCacheStats bucketCacheStats = null;
BucketAllocator bucketAllocator = null;
Expand Down Expand Up @@ -336,7 +339,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>
Expand All @@ -348,7 +363,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><% String.format("%,d", ((LruBlockCache)bc).getCurrentIndexSize()) %></td>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the only improvement that needs be done here is these should also be formatted with TraditionalBinaryPrefix.long2String().

<td>Size of INDEX Blocks</td>
</tr>
<tr>
<td>Size of Bloom Blocks</td>
<td><% String.format("%,d", ((LruBlockCache)bc).getCurrentBloomSize()) %></td>
<td>Size of BLOOM Blocks</td>
</tr>
</%if>
<& evictions_tmpl; bc = bc; &>
<& hits_tmpl; bc = bc; &>

<%if bucketCache %>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -319,8 +331,6 @@ public LruBlockCache(long maxSize, long blockSize, boolean evictionThread,
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;
Expand Down Expand Up @@ -432,7 +442,11 @@ public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory)
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()) {
Expand Down Expand Up @@ -489,8 +503,14 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean evict) {
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);
}
Expand Down Expand Up @@ -606,7 +626,12 @@ protected long evictBlock(LruCachedBlock block, boolean evictedByEvictionProcess
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) {
Expand Down Expand Up @@ -885,6 +910,14 @@ public long getCurrentDataSize() {
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();
Expand All @@ -905,6 +938,14 @@ public long getDataBlockCount() {
return this.dataBlockElements.sum();
}

public long getIndexBlockCount() {
return this.indexBlockElements.sum();
}

public long getBloomBlockCount() {
return this.bloomBlockElements.sum();
}

EvictionThread getEvictionThread() {
return this.evictionThread;
}
Expand Down