bbeaudreault commented on code in PR #4335:
URL: https://github.com/apache/hbase/pull/4335#discussion_r849461315
##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java:
##########
@@ -61,8 +62,17 @@ public CompletableFuture<List<HRegionLocation>>
getAllRegionLocations() {
return conn.registry.getMetaRegionLocations()
.thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
}
- return ClientMetaTableAccessor
- .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME),
tableName);
+ CompletableFuture<List<HRegionLocation>> future = new
CompletableFuture<>();
+ addListener(ClientMetaTableAccessor.getTableHRegionLocations(conn
+ .getTable(TableName.META_TABLE_NAME), tableName), (locs, error) -> {
+ if (error != null) {
+ future.completeExceptionally(error);
+ return;
+ }
+ locs.forEach(loc ->
conn.getLocator().getNonMetaRegionLocator().addLocationToCache(loc));
+ future.complete(locs);
Review Comment:
so the only problem with this now is, if for some reason the above line
throws an exception, we'll never complete the future. Clients would end up
hanging. We could verify this with a test where we make addLocationToCache
throw.
I think we have two options here:
- remove any future calls from here at all -- this would basically mean that
we don't care about the success of addLocationToCache. We'd be directly
returning the original future from getTableHRegionLocations. The user calling
get() on the future would finish before addLocationToCache finishes.
- wrap the above line in a try/catch and complete the future exceptionally
if an exception is thrown -- this would mean that we _do_ care about the
success of addLocationToCache. The future would only finish once the caching
has been done, and if the caching fails then the original call will be marked
failed.
The second one is more how the blocking client works and probably the more
correct for a first pass
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]