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

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

commit f6a276ab018da32f351f1047587f4f9443bdc5b0
Author: Rahul Agarkar <[email protected]>
AuthorDate: Wed Aug 2 23:06:30 2023 +0530

    HBASE-27997 Enhance prefetch executor to record region prefetch infor… 
(#5339)
    
    Signed-off-by: Wellington Chevreuil <[email protected]>
    Reviewew-by: Kota-SH <[email protected]>
---
 .../src/main/protobuf/PrefetchPersistence.proto    |  7 +++++-
 .../hadoop/hbase/io/hfile/HFilePreadReader.java    | 12 +++++++++-
 .../hadoop/hbase/io/hfile/PrefetchProtoUtils.java  | 26 +++++++++++++++++++---
 3 files changed, 40 insertions(+), 5 deletions(-)

diff --git a/hbase-protocol-shaded/src/main/protobuf/PrefetchPersistence.proto 
b/hbase-protocol-shaded/src/main/protobuf/PrefetchPersistence.proto
index d1a2b4cfd1b..a024b94baa6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/PrefetchPersistence.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/PrefetchPersistence.proto
@@ -27,5 +27,10 @@ option optimize_for = SPEED;
 
 
 message PrefetchedHfileName {
-  map<string, bool> prefetched_files = 1;
+  map<string, RegionFileSizeMap> prefetched_files = 1;
+}
+
+message RegionFileSizeMap {
+  required string region_name = 1;
+  required uint64 region_prefetch_size = 2;
 }
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 f1579ea53b8..c9768bab034 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
@@ -132,13 +132,23 @@ public class HFilePreadReader extends HFileReaderImpl {
                 LOG.warn("Close prefetch stream reader failed, path: " + path, 
e);
               }
             }
-            PrefetchExecutor.complete(path);
+            String regionName = getRegionName(path);
+            PrefetchExecutor.complete(regionName, path, offset);
           }
         }
       });
     }
   }
 
+  /*
+   * Get the region name for the given file path. A HFile is always kept under 
the <region>/<column
+   * family>/<hfile>. To find the region for a given hFile, just find the name 
of the grandparent
+   * directory.
+   */
+  private static String getRegionName(Path path) {
+    return path.getParent().getParent().getName();
+  }
+
   private static String getPathOffsetEndStr(final Path path, final long 
offset, final long end) {
     return "path=" + path.toString() + ", offset=" + offset + ", end=" + end;
   }
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchProtoUtils.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchProtoUtils.java
index e75e8a6a652..df67e4429a2 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchProtoUtils.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchProtoUtils.java
@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
+import java.util.HashMap;
 import java.util.Map;
+import org.apache.hadoop.hbase.util.Pair;
 
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.PersistentPrefetchProtos;
 
@@ -26,8 +28,26 @@ final class PrefetchProtoUtils {
   }
 
   static PersistentPrefetchProtos.PrefetchedHfileName
-    toPB(Map<String, Boolean> prefetchedHfileNames) {
-    return PersistentPrefetchProtos.PrefetchedHfileName.newBuilder()
-      .putAllPrefetchedFiles(prefetchedHfileNames).build();
+    toPB(Map<String, Pair<String, Long>> prefetchedHfileNames) {
+    Map<String, PersistentPrefetchProtos.RegionFileSizeMap> tmpMap = new 
HashMap<>();
+    prefetchedHfileNames.forEach((hFileName, regionPrefetchMap) -> {
+      PersistentPrefetchProtos.RegionFileSizeMap tmpRegionFileSize =
+        PersistentPrefetchProtos.RegionFileSizeMap.newBuilder()
+          .setRegionName(regionPrefetchMap.getFirst())
+          .setRegionPrefetchSize(regionPrefetchMap.getSecond()).build();
+      tmpMap.put(hFileName, tmpRegionFileSize);
+    });
+    return 
PersistentPrefetchProtos.PrefetchedHfileName.newBuilder().putAllPrefetchedFiles(tmpMap)
+      .build();
+  }
+
+  static Map<String, Pair<String, Long>>
+    fromPB(Map<String, PersistentPrefetchProtos.RegionFileSizeMap> 
prefetchHFileNames) {
+    Map<String, Pair<String, Long>> hFileMap = new HashMap<>();
+    prefetchHFileNames.forEach((hFileName, regionPrefetchMap) -> {
+      hFileMap.put(hFileName,
+        new Pair<>(regionPrefetchMap.getRegionName(), 
regionPrefetchMap.getRegionPrefetchSize()));
+    });
+    return hFileMap;
   }
 }

Reply via email to