virajjasani commented on a change in pull request #3215: URL: https://github.com/apache/hbase/pull/3215#discussion_r648542799
########## File path: hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java ########## @@ -158,7 +158,13 @@ public boolean containsBlock(BlockCacheKey cacheKey) { @Override public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat, boolean updateCacheMetrics) { - Cacheable value = cache.getIfPresent(cacheKey); + Cacheable value = cache.asMap().computeIfPresent(cacheKey, (blockCacheKey, cacheable) -> { + // It will be referenced by RPC path, so increase here. NOTICE: Must do the retain inside + // this block. because if retain outside the map#computeIfPresent, the evictBlock may remove + // the block and release, then we're retaining a block with refCnt=0 which is disallowed. + cacheable.retain(); + return cacheable; + }); Review comment: @saintstack @anoopsjohn @ben-manes How about this one? I am yet to benchmark this and perform chaos testing with this, but before I do it, just wanted to see if you are aligned with this rough patch. ``` 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 3e5ba1d19c..bb2b394ccd 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.util.StringUtils; +import org.apache.hbase.thirdparty.io.netty.util.IllegalReferenceCountException; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -510,14 +511,15 @@ public class LruBlockCache implements FirstLevelBlockCache { @Override public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat, boolean updateCacheMetrics) { - LruCachedBlock cb = map.computeIfPresent(cacheKey, (key, val) -> { - // It will be referenced by RPC path, so increase here. NOTICE: Must do the retain inside - // this block. because if retain outside the map#computeIfPresent, the evictBlock may remove - // the block and release, then we're retaining a block with refCnt=0 which is disallowed. - // see HBASE-22422. - val.getBuffer().retain(); - return val; - }); + LruCachedBlock cb = map.get(cacheKey); + if (cb != null) { + try { + cb.getBuffer().retain(); + } catch (IllegalReferenceCountException e) { + // map.remove(cacheKey); ==> not required here + cb = null; + } + } if (cb == null) { if (!repeat && updateCacheMetrics) { stats.miss(caching, cacheKey.isPrimary(), cacheKey.getBlockType()); ``` And this perf improvement is to be followed by all L1 caching, something we can take up as a follow up task. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org