This is an automated email from the ASF dual-hosted git repository.

wchevreuil pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new a2abf1b092c HBASE-29326 Do not prevent prefetch from running if the 
given file has already been fully cached before (#7001)
a2abf1b092c is described below

commit a2abf1b092c960cb92d5dc42fdfb4037dcc1e3d9
Author: Wellington Ramos Chevreuil <[email protected]>
AuthorDate: Tue May 27 10:28:52 2025 +0100

    HBASE-29326 Do not prevent prefetch from running if the given file has 
already been fully cached before (#7001)
    
    Signed-off-by: Tak Lon (Stephen) Wu <[email protected]>
---
 .../apache/hadoop/hbase/io/hfile/HFilePreadReader.java  | 17 +++--------------
 .../io/hfile/bucket/TestPrefetchWithBucketCache.java    | 15 ++++++++++++---
 2 files changed, 15 insertions(+), 17 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 533021a428d..996d3a85e22 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
@@ -18,8 +18,6 @@
 package org.apache.hadoop.hbase.io.hfile;
 
 import java.io.IOException;
-import java.util.Optional;
-import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
@@ -40,9 +38,8 @@ public class HFilePreadReader extends HFileReaderImpl {
     Configuration conf) throws IOException {
     super(context, fileInfo, cacheConf, conf);
     // master hosted regions, like the master procedures store wouldn't have a 
block cache
-    final MutableBoolean shouldCache = new 
MutableBoolean(cacheConf.getBlockCache().isPresent());
     // Prefetch file blocks upon open if requested
-    if (shouldCache.booleanValue() && cacheConf.shouldPrefetchOnOpen()) {
+    if (cacheConf.getBlockCache().isPresent() && 
cacheConf.shouldPrefetchOnOpen()) {
       PrefetchExecutor.request(path, new Runnable() {
         @Override
         public void run() {
@@ -50,16 +47,8 @@ public class HFilePreadReader extends HFileReaderImpl {
           long end = 0;
           HFile.Reader prefetchStreamReader = null;
           try {
-            cacheConf.getBlockCache().ifPresent(cache -> {
-              
cache.waitForCacheInitialization(WAIT_TIME_FOR_CACHE_INITIALIZATION);
-              Optional<Boolean> result = cache.shouldCacheFile(path.getName());
-              shouldCache.setValue(result.isPresent() ? 
result.get().booleanValue() : true);
-            });
-            if (!shouldCache.booleanValue()) {
-              LOG.info("Prefetch skipped for file: {}", path);
-              return;
-            }
-
+            cacheConf.getBlockCache().ifPresent(
+              cache -> 
cache.waitForCacheInitialization(WAIT_TIME_FOR_CACHE_INITIALIZATION));
             ReaderContext streamReaderContext = 
ReaderContextBuilder.newBuilder(context)
               .withReaderType(ReaderContext.ReaderType.STREAM)
               .withInputStreamWrapper(new 
FSDataInputStreamWrapper(context.getFileSystem(),
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchWithBucketCache.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchWithBucketCache.java
index a1ad1c56e74..34341ce983d 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchWithBucketCache.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestPrefetchWithBucketCache.java
@@ -57,11 +57,13 @@ import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.Cacheable;
+import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
 import org.apache.hadoop.hbase.io.hfile.PrefetchExecutor;
 import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -139,15 +141,22 @@ public class TestPrefetchWithBucketCache {
     // Our file should have 6 DATA blocks. We should wait for all of them to 
be cached
     Waiter.waitFor(conf, 300, () -> bc.getBackingMap().size() == 6);
     Map<BlockCacheKey, BucketEntry> snapshot = 
ImmutableMap.copyOf(bc.getBackingMap());
-    // Reads file again and check we are not prefetching it again
-    LOG.debug("Second read, no prefetch should happen here.");
+    LruBlockCache l1 = (LruBlockCache) ((CombinedBlockCache) 
blockCache).getFirstLevelCache();
+    assertEquals(1, l1.getBlockCount());
+    // Removes the meta block from L1 cache
+    l1.clearCache();
+    // Reads file again. Checks we are not prefetching data blocks again,
+    // but fetch back the meta block
+    LOG.debug("Second read, prefetch should run, without altering bucket cache 
state,"
+      + " only the meta block should be fetched again.");
     readStoreFile(storeFile);
-    // Makes sure the cache hasn't changed
+    // Makes sure the bucketcache entries have not changed
     snapshot.entrySet().forEach(e -> {
       BucketEntry entry = bc.getBackingMap().get(e.getKey());
       assertNotNull(entry);
       assertEquals(e.getValue().getCachedTime(), entry.getCachedTime());
     });
+    assertEquals(1, l1.getBlockCount());
     // forcibly removes first block from the bc backing map, in order to cause 
it to be cached again
     BlockCacheKey key = snapshot.keySet().stream().findFirst().get();
     LOG.debug("removing block {}", key);

Reply via email to