-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-30161 Add paginated, single-RPC RegionLocator.getRegionLocations(startKey, limit) API for bulk meta-cache warmup #8237
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
769afcf
eff0b3e
faddbe7
bdb57a5
3ef97b7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,10 +20,12 @@ | |
| import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture; | ||
| import static org.apache.hadoop.hbase.util.FutureUtils.addListener; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Arrays; | ||
| import java.util.List; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import org.apache.hadoop.hbase.ClientMetaTableAccessor; | ||
| import org.apache.hadoop.hbase.HConstants; | ||
| import org.apache.hadoop.hbase.HRegionLocation; | ||
| import org.apache.hadoop.hbase.TableName; | ||
| import org.apache.yetus.audience.InterfaceAudience; | ||
|
|
@@ -81,6 +83,39 @@ public CompletableFuture<List<HRegionLocation>> getRegionLocations(byte[] row, b | |
| .thenApply(locs -> Arrays.asList(locs.getRegionLocations())); | ||
| } | ||
|
|
||
| @Override | ||
| public CompletableFuture<List<HRegionLocation>> getRegionLocationsPage(byte[] startKey, | ||
| int limit) { | ||
| return tracedFuture(() -> { | ||
| if (TableName.isMetaTableName(tableName)) { | ||
| CompletableFuture<List<HRegionLocation>> failed = new CompletableFuture<>(); | ||
| failed.completeExceptionally( | ||
| new IOException("getRegionLocationsPage(startKey, limit) is not supported for hbase:meta;" | ||
| + " use getRegionLocation(EMPTY_START_ROW) instead.")); | ||
| return failed; | ||
| } | ||
| int effectiveLimit = limit > 0 | ||
| ? limit | ||
| : conn.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING, | ||
| HConstants.DEFAULT_HBASE_META_SCANNER_CACHING); | ||
| CompletableFuture<List<HRegionLocation>> future = | ||
| ClientMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), | ||
| tableName, startKey, effectiveLimit); | ||
| addListener(future, (locs, error) -> { | ||
| if (error != null || locs == null) { | ||
| return; | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what if there is error ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch — let me explain the intent and please push back if it's not the right call. The listener at On the listener side, if That said, I can add a if (error != null) {
LOG.debug("Skipping cache warm-up for {}: {}", tableName, error.toString());
return;
}
if (locs == null) {
return;
}Happy to add that if you think it's worth it. |
||
| for (HRegionLocation loc : locs) { | ||
| // the cache assumes that all locations have a serverName. only add if that's true | ||
| if (loc.getServerName() != null) { | ||
| conn.getLocator().getNonMetaRegionLocator().addLocationToCache(loc); | ||
| } | ||
| } | ||
| }); | ||
| return future; | ||
| }, getClass().getSimpleName() + ".getRegionLocationsPage"); | ||
| } | ||
|
|
||
| @Override | ||
| public void clearRegionLocationCache() { | ||
| conn.getLocator().clearCache(tableName); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what about merged regions ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for flagging — let me share my understanding and please correct me if I've got this wrong.
I think merge parents don't need filtering here because of how splits vs merges record cleanup state in
hbase:meta:Split: the parent row stays around with
setOffline(true).setSplit(true)(seeRegionStateStore#splitRegion) and carriesinfo:splitA/info:splitBpointers. The catalog janitor seems to use that row to track whether daughters still hold HFile references, and only deletes the parent row + HDFS dir once references are gone. So during that window, meta scans do see the parent — which is whyexcludeOfflinedSplitParentsexists.Merge: from what I can see in
RegionStateStore#mergeRegions, the same multi-mutate that commits the merge deletes every parent row and writes a single child row withinfo:merge_0000,info:merge_0001, … pointing back at the parents. So the "waiting on HFile cleanup" bookkeeping lives on the child row, not on the parents.If that's right, a meta scan should never see merge parents, and the merged child row is itself the live region for that key range — which is what we'd want to return. This also seems to match
getAllRegionLocations()'s existing behavior; both shareCollectRegionLocationsVisitor(excludeOfflinedSplitParents=true).Does this line up with your understanding? Happy to add a comment near the visitor call site noting this if it'd help future readers, or revisit the filter if I'm missing a case.