Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Original file line number Diff line number Diff line change
Expand Up @@ -147,13 +147,23 @@ public void run() {
LOG.warn("Close prefetch stream reader failed, path: " + path, e);
}
}
String regionName = getRegionName(path);
PrefetchExecutor.complete(path);
}
}
});
}
}

/*
* 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;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -26,8 +28,26 @@ private 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;
}
}