From a2a1d88a3409cf8b0263f411d24e9fbef25a6235 Mon Sep 17 00:00:00 2001 From: Janardhan Hungund Date: Wed, 23 Oct 2024 17:59:41 +0530 Subject: [PATCH] HBASE-28934: Avoid waits within HFilePreadReader constructors for cache initialisation. The current implementation is such that the HFilePreadReader constructor waits for the cache initialisation to decide whether or not, to create a prefetcher for an HFile. However, as the HFilePreadReader constructor waits for the bucket cache initialisation, it blocks the region from bein opened. Any calls to read from the region fail until the cache is initialised. This prevents the region from being usable, which otherwise, can read from the data store (cloud storage) to serve the client requests. This change prevents the waits with the HFilePreadReader constructor and instead, move this wait to the prefetch thread, which waits until the cache is initialised to decide whether or, not to prefetch the file. Any requests to the regions will be served from the main data store instead of cache. Also, the missed cache access during the cache initialisation should be accounted for the cache misses. Change-Id: I952c592a1020874ad874f1909f776d8652375a1b --- .../hbase/io/hfile/HFilePreadReader.java | 20 +++++++++++-------- .../hbase/io/hfile/bucket/BucketCache.java | 1 + 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java index 8c9d473b53be..bfa807d4a6f9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java @@ -39,23 +39,27 @@ public class HFilePreadReader extends HFileReaderImpl { public HFilePreadReader(ReaderContext context, HFileInfo fileInfo, CacheConfig cacheConf, Configuration conf) throws IOException { super(context, fileInfo, cacheConf, conf); - final MutableBoolean shouldCache = new MutableBoolean(true); - - cacheConf.getBlockCache().ifPresent(cache -> { - cache.waitForCacheInitialization(WAIT_TIME_FOR_CACHE_INITIALIZATION); - Optional result = cache.shouldCacheFile(path.getName()); - shouldCache.setValue(result.isPresent() ? result.get().booleanValue() : true); - }); // Prefetch file blocks upon open if requested - if (cacheConf.shouldPrefetchOnOpen() && shouldCache.booleanValue()) { + if (cacheConf.shouldPrefetchOnOpen()) { PrefetchExecutor.request(path, new Runnable() { @Override public void run() { long offset = 0; long end = 0; HFile.Reader prefetchStreamReader = null; + final MutableBoolean shouldCache = new MutableBoolean(true); try { + cacheConf.getBlockCache().ifPresent(cache -> { + cache.waitForCacheInitialization(WAIT_TIME_FOR_CACHE_INITIALIZATION); + Optional result = cache.shouldCacheFile(path.getName()); + shouldCache.setValue(result.isPresent() ? result.get().booleanValue() : true); + }); + if (!shouldCache.booleanValue()) { + LOG.info("Prefetch skipped for the file: " + path.getName()); + return; + } + ReaderContext streamReaderContext = ReaderContextBuilder.newBuilder(context) .withReaderType(ReaderContext.ReaderType.STREAM) .withInputStreamWrapper(new FSDataInputStreamWrapper(context.getFileSystem(), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index bc7bc955864c..4cd50ced57f1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -651,6 +651,7 @@ public BucketEntry getBlockForReference(BlockCacheKey key) { public Cacheable getBlock(BlockCacheKey key, boolean caching, boolean repeat, boolean updateCacheMetrics) { if (!isCacheEnabled()) { + cacheStats.miss(caching, key.isPrimary(), key.getBlockType()); return null; } RAMQueueEntry re = ramCache.get(key);