diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java index 494a588aadb8..e1bbd10d75a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/LruAdaptiveBlockCache.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 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; @@ -222,15 +223,7 @@ public class LruAdaptiveBlockCache implements FirstLevelBlockCache { = "hbase.lru.cache.heavy.eviction.overhead.coefficient"; private static final float DEFAULT_LRU_CACHE_HEAVY_EVICTION_OVERHEAD_COEFFICIENT = 0.01f; - /** - * Defined the cache map as {@link ConcurrentHashMap} here, because in - * {@link LruAdaptiveBlockCache#getBlock}, we need to guarantee the atomicity - * of map#computeIfPresent (key, func). Besides, the func method must execute exactly once only - * when the key is present and under the lock context, otherwise the reference count will be - * messed up. Notice that the - * {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that. - */ - private transient final ConcurrentHashMap map; + private transient final Map map; /** Eviction lock (locked when eviction in process) */ private transient final ReentrantLock evictionLock = new ReentrantLock(true); @@ -646,14 +639,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean evict) { @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) { + cb = null; + LOG.debug("AdaptiveLRU cache block retain caused refCount Exception. Treating this as L1" + + " cache miss. Exception: {}", e.getMessage()); + } + } if (cb == null) { if (!repeat && updateCacheMetrics) { stats.miss(caching, cacheKey.isPrimary(), cacheKey.getBlockType()); 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 3e5ba1d19c56..98f738aa902a 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.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; @@ -145,14 +146,7 @@ public class LruBlockCache implements FirstLevelBlockCache { private static final String LRU_MAX_BLOCK_SIZE = "hbase.lru.max.block.size"; private static final long DEFAULT_MAX_BLOCK_SIZE = 16L * 1024L * 1024L; - /** - * Defined the cache map as {@link ConcurrentHashMap} here, because in - * {@link LruBlockCache#getBlock}, we need to guarantee the atomicity of map#computeIfPresent - * (key, func). Besides, the func method must execute exactly once only when the key is present - * and under the lock context, otherwise the reference count will be messed up. Notice that the - * {@link java.util.concurrent.ConcurrentSkipListMap} can not guarantee that. - */ - private transient final ConcurrentHashMap map; + private transient final Map map; /** Eviction lock (locked when eviction in process) */ private transient final ReentrantLock evictionLock = new ReentrantLock(true); @@ -510,14 +504,16 @@ private long updateSizeMetrics(LruCachedBlock cb, boolean evict) { @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) { + cb = null; + LOG.debug("LRU cache block retain caused refCount Exception. Treating this as L1 cache" + + " miss. Exception: {}", e.getMessage()); + } + } if (cb == null) { if (!repeat && updateCacheMetrics) { stats.miss(caching, cacheKey.isPrimary(), cacheKey.getBlockType()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java index 6f914f56a653..33aca6356dd8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/TinyLfuBlockCache.java @@ -38,6 +38,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hbase.thirdparty.io.netty.util.IllegalReferenceCountException; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -158,13 +159,16 @@ public boolean containsBlock(BlockCacheKey cacheKey) { @Override public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat, boolean updateCacheMetrics) { - 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; - }); + Cacheable value = cache.getIfPresent(cacheKey); + if (value != null) { + try { + value.retain(); + } catch (IllegalReferenceCountException e) { + value = null; + LOG.debug("TinyLfu cache block retain caused refCount Exception. Treating this as L1 cache" + + " miss. Exception: {}", e.getMessage()); + } + } if (value == null) { if (repeat) { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index e58f08c76e8a..e639078a5dcb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -910,7 +910,7 @@ public void testReaderWithAdaptiveLruCombinedBlockCache() throws Exception { } /** - * Test case for CombinedBlockCache with AdaptiveLRU as L1 cache + * Test case for CombinedBlockCache with LRU as L1 cache */ @Test public void testReaderWithLruCombinedBlockCache() throws Exception {