HBASE-27867 Close the L1 victim handler race (#5239)

When we evict a block from L1 and move it to L2 there is a brief window of
time where we have removed the block from the L1 map and yet the victim
handler has not completed execution. Some read-your-write use cases can be
significantly impacted even though the window is small.

Victim handling can be made atomic with respect to the unmapping operation.
The upside is there will be no L1+L2 misses during the transition. The
downside is if the victim handler takes a long time to execute – currently
they are all very fast, so only a theoretical risk – then other removals or
insertions in L1 can block until it completes.

Signed-off-by: Duo Zhang <zhangduo@apache.org>
Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Andrew Purtell 2023-05-25 16:46:15 -07:00
parent 24160a45ae
commit b6d063550a
1 changed files with 27 additions and 13 deletions

View File

@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.lang3.mutable.MutableBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -147,10 +148,12 @@ public class LruBlockCache implements FirstLevelBlockCache {
/**
* 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.
* {@link LruBlockCache#getBlock}, we need to guarantee the atomicity of map#k (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. Some code using
* #computeIfPresent also expects the supplier to be executed only once. ConcurrentHashMap can
* guarantee that. Other types may not.
*/
private transient final ConcurrentHashMap<BlockCacheKey, LruCachedBlock> map;
@ -511,6 +514,7 @@ public class LruBlockCache implements FirstLevelBlockCache {
@Override
public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat,
boolean updateCacheMetrics) {
// Note: 'map' must be a ConcurrentHashMap or the supplier may be invoked more than once.
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
@ -592,10 +596,27 @@ public class LruBlockCache implements FirstLevelBlockCache {
* @return the heap size of evicted block
*/
protected long evictBlock(LruCachedBlock block, boolean evictedByEvictionProcess) {
LruCachedBlock previous = map.remove(block.getCacheKey());
if (previous == null) {
final MutableBoolean evicted = new MutableBoolean(false);
// Note: 'map' must be a ConcurrentHashMap or the supplier may be invoked more than once.
map.computeIfPresent(block.getCacheKey(), (k, v) -> {
// Run the victim handler before we remove the mapping in the L1 map. It must complete
// quickly because other removal or insertion operations can be blocked in the meantime.
if (evictedByEvictionProcess && victimHandler != null) {
victimHandler.cacheBlock(k, v.getBuffer());
}
// Decrease the block's reference count, and if refCount is 0, then it'll auto-deallocate. DO
// NOT move this up because if we do that then the victimHandler may access the buffer with
// refCnt = 0 which is disallowed.
v.getBuffer().release();
evicted.setTrue();
// By returning null from the supplier we remove the mapping from the L1 map.
return null;
});
// If we didn't find anything to evict there is nothing more to do here.
if (evicted.isFalse()) {
return 0;
}
// We evicted the block so update L1 statistics.
updateSizeMetrics(block, true);
long val = elements.decrementAndGet();
if (LOG.isTraceEnabled()) {
@ -614,14 +635,7 @@ public class LruBlockCache implements FirstLevelBlockCache {
// When the eviction of the block happened because of invalidation of HFiles, no need to
// update the stats counter.
stats.evicted(block.getCachedTime(), block.getCacheKey().isPrimary());
if (victimHandler != null) {
victimHandler.cacheBlock(block.getCacheKey(), block.getBuffer());
}
}
// Decrease the block's reference count, and if refCount is 0, then it'll auto-deallocate. DO
// NOT move this up because if do that then the victimHandler may access the buffer with
// refCnt = 0 which is disallowed.
previous.getBuffer().release();
return block.heapSize();
}