Skip to content

Commit 8b1f690

Browse files
authored
HBASE-27794: Tooling for parsing/reading the prefetch files list file (#5468) (#5491)
Signed-off-by: Wellington Chevreuil <[email protected]> (cherry picked from commit fa4c896)
1 parent 0d04a60 commit 8b1f690

File tree

17 files changed

+143
-7
lines changed

17 files changed

+143
-7
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3317,4 +3317,9 @@ List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, Server
33173317
* @throws IOException if a remote or network exception occurs
33183318
*/
33193319
Future<Void> truncateRegionAsync(byte[] regionName) throws IOException;
3320+
3321+
/**
3322+
* Get the list of cached files
3323+
*/
3324+
List<String> getCachedFilesList(ServerName serverName) throws IOException;
33203325
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1702,4 +1702,9 @@ CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, Str
17021702
* Flush master local region
17031703
*/
17041704
CompletableFuture<Void> flushMasterStore();
1705+
1706+
/**
1707+
* Get the list of cached files
1708+
*/
1709+
CompletableFuture<List<String>> getCachedFilesList(ServerName serverName);
17051710
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -911,4 +911,9 @@ public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNam
911911
public CompletableFuture<Void> flushMasterStore() {
912912
return wrap(rawAdmin.flushMasterStore());
913913
}
914+
915+
@Override
916+
public CompletableFuture<List<String>> getCachedFilesList(ServerName serverName) {
917+
return wrap(rawAdmin.getCachedFilesList(serverName));
918+
}
914919
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2078,14 +2078,23 @@ public Future<Void> truncateRegionAsync(byte[] regionName) throws IOException {
20782078
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
20792079
}
20802080

2081-
TableName tableName = hri.getTable();
2081+
TableName tableName = (hri != null) ? hri.getTable() : null;
20822082

20832083
MasterProtos.TruncateRegionResponse response =
20842084
executeCallable(getTruncateRegionCallable(tableName, hri));
20852085

20862086
return new TruncateRegionFuture(this, tableName, response);
20872087
}
20882088

2089+
/**
2090+
* Get the list of cached files
2091+
*/
2092+
@Override
2093+
public List<String> getCachedFilesList(ServerName serverName) throws IOException {
2094+
return ProtobufUtil.getCachedFilesList(rpcControllerFactory.newController(),
2095+
this.connection.getAdmin(serverName));
2096+
}
2097+
20892098
private MasterCallable<MasterProtos.TruncateRegionResponse>
20902099
getTruncateRegionCallable(TableName tableName, RegionInfo hri) {
20912100
return new MasterCallable<MasterProtos.TruncateRegionResponse>(getConnection(),

hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -129,6 +129,8 @@
129129
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
130130
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
131131
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
132+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest;
133+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse;
132134
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
133135
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
134136
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
@@ -4264,4 +4266,15 @@ Void> call(controller, stub, request.build(),
42644266
(s, c, req, done) -> s.flushMasterStore(c, req, done), resp -> null))
42654267
.call();
42664268
}
4269+
4270+
@Override
4271+
public CompletableFuture<List<String>> getCachedFilesList(ServerName serverName) {
4272+
GetCachedFilesListRequest.Builder request = GetCachedFilesListRequest.newBuilder();
4273+
return this.<List<String>> newAdminCaller()
4274+
.action((controller, stub) -> this.<GetCachedFilesListRequest, GetCachedFilesListResponse,
4275+
List<String>> adminCall(controller, stub, request.build(),
4276+
(s, c, req, done) -> s.getCachedFilesList(c, req, done),
4277+
resp -> resp.getCachedFilesList()))
4278+
.serverName(serverName).call();
4279+
}
42674280
}

hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -149,6 +149,8 @@
149149
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
150150
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses;
151151
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
152+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest;
153+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse;
152154
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
153155
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
154156
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
@@ -1746,6 +1748,21 @@ public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(
17461748
return getRegionInfos(response);
17471749
}
17481750

1751+
/**
1752+
* Get the list of cached files
1753+
*/
1754+
public static List<String> getCachedFilesList(final RpcController controller,
1755+
final AdminService.BlockingInterface admin) throws IOException {
1756+
GetCachedFilesListRequest request = GetCachedFilesListRequest.newBuilder().build();
1757+
GetCachedFilesListResponse response = null;
1758+
try {
1759+
response = admin.getCachedFilesList(controller, request);
1760+
} catch (ServiceException se) {
1761+
throw getRemoteException(se);
1762+
}
1763+
return new ArrayList<>(response.getCachedFilesList());
1764+
}
1765+
17491766
/**
17501767
* Get the list of region info from a GetOnlineRegionResponse
17511768
* @param proto the GetOnlineRegionResponse

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -283,6 +283,13 @@ message ExecuteProceduresRequest {
283283
message ExecuteProceduresResponse {
284284
}
285285

286+
message GetCachedFilesListRequest {
287+
}
288+
289+
message GetCachedFilesListResponse {
290+
repeated string cached_files = 1;
291+
}
292+
286293
/**
287294
* Slow/Large log (LogRequest) use-case specific RPC request. This request payload will be
288295
* converted in bytes and sent to generic RPC API: GetLogEntries
@@ -406,4 +413,7 @@ service AdminService {
406413
rpc GetLogEntries(LogRequest)
407414
returns(LogEntry);
408415

416+
rpc GetCachedFilesList(GetCachedFilesListRequest)
417+
returns(GetCachedFilesListResponse);
418+
409419
}

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

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
package org.apache.hadoop.hbase.io.hfile;
1919

2020
import java.util.Iterator;
21+
import java.util.Map;
22+
import java.util.Optional;
2123
import org.apache.yetus.audience.InterfaceAudience;
2224

2325
/**
@@ -146,4 +148,11 @@ Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat,
146148
default boolean isMetaBlock(BlockType blockType) {
147149
return blockType != null && blockType.getCategory() != BlockType.BlockCategory.DATA;
148150
}
151+
152+
/**
153+
* Returns the list of fully cached files
154+
*/
155+
default Optional<Map<String, Boolean>> getFullyCachedFiles() {
156+
return Optional.empty();
157+
}
149158
}

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

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,8 @@
1818
package org.apache.hadoop.hbase.io.hfile;
1919

2020
import java.util.Iterator;
21+
import java.util.Map;
22+
import java.util.Optional;
2123
import org.apache.hadoop.hbase.io.HeapSize;
2224
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
2325
import org.apache.yetus.audience.InterfaceAudience;
@@ -369,6 +371,14 @@ public BlockCache[] getBlockCaches() {
369371
return new BlockCache[] { this.l1Cache, this.l2Cache };
370372
}
371373

374+
/**
375+
* Returns the list of fully cached files
376+
*/
377+
@Override
378+
public Optional<Map<String, Boolean>> getFullyCachedFiles() {
379+
return this.l2Cache.getFullyCachedFiles();
380+
}
381+
372382
@Override
373383
public void setMaxSize(long size) {
374384
this.l1Cache.setMaxSize(size);

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -42,8 +42,9 @@ public HFilePreadReader(ReaderContext context, HFileInfo fileInfo, CacheConfig c
4242
final MutableBoolean fileAlreadyCached = new MutableBoolean(false);
4343
Optional<BucketCache> bucketCacheOptional =
4444
BucketCache.getBucketCacheFromCacheConfig(cacheConf);
45-
bucketCacheOptional.ifPresent(bc -> fileAlreadyCached
46-
.setValue(bc.getFullyCachedFiles().get(path.getName()) == null ? false : true));
45+
bucketCacheOptional.flatMap(BucketCache::getFullyCachedFiles).ifPresent(fcf -> {
46+
fileAlreadyCached.setValue(fcf.get(path.getName()) == null ? false : true);
47+
});
4748
// Prefetch file blocks upon open if requested
4849
if (
4950
cacheConf.shouldPrefetchOnOpen() && cacheIfCompactionsOff()

0 commit comments

Comments
 (0)