frostruan commented on code in PR #4335:
URL: https://github.com/apache/hbase/pull/4335#discussion_r849526294
##########
hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java:
##########
@@ -460,4 +464,35 @@ public void testConcurrentUpdateCachedLocationOnError()
throws Exception {
IntStream.range(0, 100).parallel()
.forEach(i -> locator.updateCachedLocationOnError(loc, new
NotServingRegionException()));
}
+
+ @Test
+ public void testCacheLocationWhenGetAllLocations() throws Exception {
+ createMultiRegionTable();
+ AsyncConnectionImpl conn = (AsyncConnectionImpl)
+
ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+ conn.getRegionLocator(TABLE_NAME).getAllRegionLocations().get();
+ List<RegionInfo> regions = TEST_UTIL.getAdmin().getRegions(TABLE_NAME);
+ for (RegionInfo region : regions) {
+ assertNotNull(conn.getLocator().getRegionLocationInCache(TABLE_NAME,
region.getStartKey()));
+ }
+ }
+
+ @Test
+ public void testCacheLocationExceptionallyWhenGetAllLocations() throws
Exception {
+ createMultiRegionTable();
+ AsyncConnectionImpl conn = (AsyncConnectionImpl)
+
ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
+ AsyncConnectionImpl spyConn = Mockito.spy(conn);
+ Mockito.when(spyConn.getTable(TableName.META_TABLE_NAME))
Review Comment:
Yes, you are right, this exception **do** does not complete the future.
Didn't notice that before, thanks for pointing it out.
I plan to mock the meta table and throws an exception when we scan it. I
think it is deep enough to get an exceptional future.
Also, I am a little confused about what you say "it might be enough to have
conn.getLocator().getRegionLocations() return an exceptional future". From my
understanding, we will not call this method. Would you mind explaining a little
more ?
##########
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:
Agree. We should avoid the potential hung risk. I'll fix this. Thanks.
--
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]