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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 3deb8fa921d HBASE-28953 Prefetch thread shouldn't run for master store 
(#6438) (#6445)
3deb8fa921d is described below

commit 3deb8fa921d1fcc4c2c5e30f5c306716d3eb1ae9
Author: Wellington Ramos Chevreuil <[email protected]>
AuthorDate: Fri Nov 15 11:40:39 2024 +0000

    HBASE-28953 Prefetch thread shouldn't run for master store (#6438) (#6445)
    
    Signed-off-by: Peter Somogyi <[email protected]>
    Signed-off-by: Ankit Singhal <[email protected]>
    Signed-off-by: Pankaj Kumar<[email protected]>
---
 .../hadoop/hbase/io/hfile/HFilePreadReader.java    |  8 ++++----
 .../apache/hadoop/hbase/io/hfile/TestPrefetch.java | 22 ++++++++++++++++++++++
 2 files changed, 26 insertions(+), 4 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 bfa807d4a6f..6266be6cbcc 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,16 +39,16 @@ public class HFilePreadReader extends HFileReaderImpl {
   public HFilePreadReader(ReaderContext context, HFileInfo fileInfo, 
CacheConfig cacheConf,
     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 (cacheConf.shouldPrefetchOnOpen()) {
+    if (shouldCache.booleanValue() && 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);
@@ -56,7 +56,7 @@ public class HFilePreadReader extends HFileReaderImpl {
               shouldCache.setValue(result.isPresent() ? 
result.get().booleanValue() : true);
             });
             if (!shouldCache.booleanValue()) {
-              LOG.info("Prefetch skipped for the file: " + path.getName());
+              LOG.info("Prefetch skipped for file: {}", path);
               return;
             }
 
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index 83f10f4a61c..d27995b8725 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -381,6 +381,28 @@ public class TestPrefetch {
     prefetchExecutorNotifier.onConfigurationChange(conf);
   }
 
+  @Test
+  public void testPrefetchWhenNoBlockCache() throws Exception {
+    PrefetchExecutorNotifier prefetchExecutorNotifier = new 
PrefetchExecutorNotifier(conf);
+    try {
+      // Set a delay to max, as we don't need to have the thread running, but 
rather
+      // assert that it never gets scheduled
+      conf.setInt(PREFETCH_DELAY, Integer.MAX_VALUE);
+      conf.setFloat(PREFETCH_DELAY_VARIATION, 0.0f);
+      prefetchExecutorNotifier.onConfigurationChange(conf);
+
+      HFileContext context = new 
HFileContextBuilder().withCompression(Compression.Algorithm.GZ)
+        .withBlockSize(DATA_BLOCK_SIZE).build();
+      Path storeFile = writeStoreFile("testPrefetchWhenNoBlockCache", context);
+      HFile.createReader(fs, storeFile, new CacheConfig(conf), true, conf);
+      assertEquals(0, PrefetchExecutor.getPrefetchFutures().size());
+    } finally {
+      conf.setInt(PREFETCH_DELAY, 1000);
+      conf.setFloat(PREFETCH_DELAY_VARIATION, 
PREFETCH_DELAY_VARIATION_DEFAULT_VALUE);
+      prefetchExecutorNotifier.onConfigurationChange(conf);
+    }
+  }
+
   @Test
   public void testPrefetchDoesntSkipHFileLink() throws Exception {
     testPrefetchWhenHFileLink(c -> {

Reply via email to