Skip to content

Commit 497ffe2

Browse files
Kota-SHwchevreuil
authored andcommitted
HBASE-27794: Tooling for parsing/reading the prefetch files list file (apache#5468)
Signed-off-by: Wellingt
1 parent e77c1c3 commit 497ffe2

File tree

18 files changed

+142
-6
lines changed

18 files changed

+142
-6
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
@@ -2629,4 +2629,9 @@ List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, Server
26292629
* Flush master local region
26302630
*/
26312631
void flushMasterStore() throws IOException;
2632+
2633+
/**
2634+
* Get the list of cached files
2635+
*/
2636+
List<String> getCachedFilesList(ServerName serverName) throws IOException;
26322637
}

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

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1125,4 +1125,9 @@ public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
11251125
public void flushMasterStore() throws IOException {
11261126
get(admin.flushMasterStore());
11271127
}
1128+
1129+
@Override
1130+
public List<String> getCachedFilesList(ServerName serverName) throws IOException {
1131+
return get(admin.getCachedFilesList(serverName));
1132+
}
11281133
}

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
@@ -1843,4 +1843,9 @@ CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, Str
18431843
* Flush master local region
18441844
*/
18451845
CompletableFuture<Void> flushMasterStore();
1846+
1847+
/**
1848+
* Get the list of cached files
1849+
*/
1850+
CompletableFuture<List<String>> getCachedFilesList(ServerName serverName);
18461851
}

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
@@ -995,4 +995,9 @@ public CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNam
995995
public CompletableFuture<Void> flushMasterStore() {
996996
return wrap(rawAdmin.flushMasterStore());
997997
}
998+
999+
@Override
1000+
public CompletableFuture<List<String>> getCachedFilesList(ServerName serverName) {
1001+
return wrap(rawAdmin.getCachedFilesList(serverName));
1002+
}
9981003
}

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
@@ -132,6 +132,8 @@
132132
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactionSwitchResponse;
133133
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionRequest;
134134
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.FlushRegionResponse;
135+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest;
136+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse;
135137
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
136138
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
137139
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
@@ -4519,4 +4521,15 @@ Void> call(controller, stub, request.build(),
45194521
(s, c, req, done) -> s.flushMasterStore(c, req, done), resp -> null))
45204522
.call();
45214523
}
4524+
4525+
@Override
4526+
public CompletableFuture<List<String>> getCachedFilesList(ServerName serverName) {
4527+
GetCachedFilesListRequest.Builder request = GetCachedFilesListRequest.newBuilder();
4528+
return this.<List<String>> newAdminCaller()
4529+
.action((controller, stub) -> this.<GetCachedFilesListRequest, GetCachedFilesListResponse,
4530+
List<String>> adminCall(controller, stub, request.build(),
4531+
(s, c, req, done) -> s.getCachedFilesList(c, req, done),
4532+
resp -> resp.getCachedFilesList()))
4533+
.serverName(serverName).call();
4534+
}
45224535
}

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
@@ -153,6 +153,8 @@
153153
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
154154
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ClearSlowLogResponses;
155155
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionRequest;
156+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListRequest;
157+
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetCachedFilesListResponse;
156158
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
157159
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
158160
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
@@ -1780,6 +1782,21 @@ public static List<org.apache.hadoop.hbase.client.RegionInfo> getOnlineRegions(
17801782
return getRegionInfos(response);
17811783
}
17821784

1785+
/**
1786+
* Get the list of cached files
1787+
*/
1788+
public static List<String> getCachedFilesList(final RpcController controller,
1789+
final AdminService.BlockingInterface admin) throws IOException {
1790+
GetCachedFilesListRequest request = GetCachedFilesListRequest.newBuilder().build();
1791+
GetCachedFilesListResponse response = null;
1792+
try {
1793+
response = admin.getCachedFilesList(controller, request);
1794+
} catch (ServiceException se) {
1795+
throw getRemoteException(se);
1796+
}
1797+
return new ArrayList<>(response.getCachedFilesList());
1798+
}
1799+
17831800
/**
17841801
* Get the list of region info from a GetOnlineRegionResponse
17851802
* @param proto the GetOnlineRegionResponse

hbase-protocol-shaded/src/main/protobuf/server/region/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
@@ -405,4 +412,7 @@ service AdminService {
405412
rpc GetLogEntries(LogRequest)
406413
returns(LogEntry);
407414

415+
rpc GetCachedFilesList(GetCachedFilesListRequest)
416+
returns(GetCachedFilesListResponse);
417+
408418
}

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

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

2020
import java.util.Iterator;
21+
import java.util.Map;
22+
import java.util.Optional;
23+
import org.apache.hadoop.hbase.util.Pair;
2124
import org.apache.yetus.audience.InterfaceAudience;
2225

2326
/**
@@ -161,4 +164,11 @@ default Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repe
161164
default boolean isMetaBlock(BlockType blockType) {
162165
return blockType != null && blockType.getCategory() != BlockType.BlockCategory.DATA;
163166
}
167+
168+
/**
169+
* Returns the list of fully cached files
170+
*/
171+
default Optional<Map<String, Pair<String, Long>>> getFullyCachedFiles() {
172+
return Optional.empty();
173+
}
164174
}

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

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,8 +18,11 @@
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;
25+
import org.apache.hadoop.hbase.util.Pair;
2326
import org.apache.yetus.audience.InterfaceAudience;
2427

2528
/**
@@ -382,6 +385,14 @@ public BlockCache[] getBlockCaches() {
382385
return new BlockCache[] { this.l1Cache, this.l2Cache };
383386
}
384387

388+
/**
389+
* Returns the list of fully cached files
390+
*/
391+
@Override
392+
public Optional<Map<String, Pair<String, Long>>> getFullyCachedFiles() {
393+
return this.l2Cache.getFullyCachedFiles();
394+
}
395+
385396
@Override
386397
public void setMaxSize(long size) {
387398
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)