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

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


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

commit 524ba0124c427d8637f47327421a195527e13cbb
Author: Wellington Ramos Chevreuil <[email protected]>
AuthorDate: Tue May 27 20:05:23 2025 +0100

    HBASE-29326 Do not prevent prefetch from running if the given file has 
already been fully cached before (#7001) (#7038)
    
    Signed-off-by: Tak Lon (Stephen) Wu <[email protected]>
    Change-Id: Idec460743b27295855de181e0390f4da632085d4
---
 .../apache/hadoop/hbase/io/hfile/HFilePreadReader.java    | 15 +++------------
 .../hbase/io/hfile/TestPrefetchWithBucketCache.java       | 13 ++++++++++---
 2 files changed, 13 insertions(+), 15 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 92623731482..08649ebb315 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;
@@ -38,15 +36,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());
-
-    cacheConf.getBlockCache().ifPresent(cache -> {
-      Optional<Boolean> result = cache.shouldCacheFile(path.getName());
-      shouldCache.setValue(result.isPresent() ? result.get().booleanValue() : 
true);
-    });
-
     // Prefetch file blocks upon open if requested
-    if (cacheConf.shouldPrefetchOnOpen() && cacheIfCompactionsOff() && 
shouldCache.booleanValue()) {
+    if (cacheConf.getBlockCache().isPresent() && 
cacheConf.shouldPrefetchOnOpen()) {
       PrefetchExecutor.request(path, new Runnable() {
         @Override
         public void run() {
@@ -134,8 +125,8 @@ public class HFilePreadReader extends HFileReaderImpl {
             }
           } catch (IOException e) {
             // IOExceptions are probably due to region closes (relocation, 
etc.)
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Prefetch " + getPathOffsetEndStr(path, offset, end), 
e);
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Prefetch " + getPathOffsetEndStr(path, offset, end), 
e);
             }
           } catch (Throwable e) {
             // Other exceptions are interesting
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchWithBucketCache.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchWithBucketCache.java
index db8f2213d0c..8e50822a90a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchWithBucketCache.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetchWithBucketCache.java
@@ -114,15 +114,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