Skip to content

Commit 2f747d4

Browse files
ragarkarwchevreuil
authored andcommitted
HBASE-27997 Enhance prefetch executor to record region prefetch infor… (apache#5339)
Signed-off-by: Wellington Chevreuil <[email protected]> Reviewew-by: Kota-SH <[email protected]>
1 parent 94ed6ad commit 2f747d4

File tree

3 files changed

+38
-4
lines changed

3 files changed

+38
-4
lines changed

hbase-protocol-shaded/src/main/protobuf/PrefetchPersistence.proto

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,5 +27,10 @@ option optimize_for = SPEED;
2727

2828

2929
message PrefetchedHfileName {
30-
map<string, bool> prefetched_files = 1;
30+
map<string, RegionFileSizeMap> prefetched_files = 1;
31+
}
32+
33+
message RegionFileSizeMap {
34+
required string region_name = 1;
35+
required uint64 region_prefetch_size = 2;
3136
}

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePreadReader.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -137,6 +137,15 @@ public void run() {
137137
}
138138
}
139139

140+
/*
141+
* Get the region name for the given file path. A HFile is always kept under the <region>/<column
142+
* family>/<hfile>. To find the region for a given hFile, just find the name of the grandparent
143+
* directory.
144+
*/
145+
private static String getRegionName(Path path) {
146+
return path.getParent().getParent().getName();
147+
}
148+
140149
private static String getPathOffsetEndStr(final Path path, final long offset, final long end) {
141150
return "path=" + path.toString() + ", offset=" + offset + ", end=" + end;
142151
}

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchProtoUtils.java

Lines changed: 23 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,9 @@
1717
*/
1818
package org.apache.hadoop.hbase.io.hfile;
1919

20+
import java.util.HashMap;
2021
import java.util.Map;
22+
import org.apache.hadoop.hbase.util.Pair;
2123

2224
import org.apache.hadoop.hbase.shaded.protobuf.generated.PersistentPrefetchProtos;
2325

@@ -26,8 +28,26 @@ private PrefetchProtoUtils() {
2628
}
2729

2830
static PersistentPrefetchProtos.PrefetchedHfileName
29-
toPB(Map<String, Boolean> prefetchedHfileNames) {
30-
return PersistentPrefetchProtos.PrefetchedHfileName.newBuilder()
31-
.putAllPrefetchedFiles(prefetchedHfileNames).build();
31+
toPB(Map<String, Pair<String, Long>> prefetchedHfileNames) {
32+
Map<String, PersistentPrefetchProtos.RegionFileSizeMap> tmpMap = new HashMap<>();
33+
prefetchedHfileNames.forEach((hFileName, regionPrefetchMap) -> {
34+
PersistentPrefetchProtos.RegionFileSizeMap tmpRegionFileSize =
35+
PersistentPrefetchProtos.RegionFileSizeMap.newBuilder()
36+
.setRegionName(regionPrefetchMap.getFirst())
37+
.setRegionPrefetchSize(regionPrefetchMap.getSecond()).build();
38+
tmpMap.put(hFileName, tmpRegionFileSize);
39+
});
40+
return PersistentPrefetchProtos.PrefetchedHfileName.newBuilder().putAllPrefetchedFiles(tmpMap)
41+
.build();
42+
}
43+
44+
static Map<String, Pair<String, Long>>
45+
fromPB(Map<String, PersistentPrefetchProtos.RegionFileSizeMap> prefetchHFileNames) {
46+
Map<String, Pair<String, Long>> hFileMap = new HashMap<>();
47+
prefetchHFileNames.forEach((hFileName, regionPrefetchMap) -> {
48+
hFileMap.put(hFileName,
49+
new Pair<>(regionPrefetchMap.getRegionName(), regionPrefetchMap.getRegionPrefetchSize()));
50+
});
51+
return hFileMap;
3252
}
3353
}

0 commit comments

Comments
 (0)