sanjeet006py commented on code in PR #8237:
URL: https://github.com/apache/hbase/pull/8237#discussion_r3249815297


##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java:
##########
@@ -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;
+        }

Review Comment:
   Good catch — let me explain the intent and please push back if it's not the 
right call.
   
   The listener at `AsyncTableRegionLocatorImpl.java:104` is a side-effect-only 
hook for cache warm-up. The `future` returned to the caller is the one from 
`ClientMetaTableAccessor.getTableHRegionLocations`, which already completes 
exceptionally on any meta-scan error (see `getTableRegionsAndLocations`'s 
`future.completeExceptionally(error)` paths). So the error propagates to the 
caller through the returned future regardless of what the listener does.
   
   On the listener side, if `error != null` there's simply nothing valid to 
cache, and if `locs == null` there's nothing to iterate. Either way, returning 
early is the safe behavior — we don't want to mask the error or double-fail the 
future, just skip cache population for this call. The next call to 
`getRegionLocationsPage` (or any locator path) will populate normally.
   
   That said, I can add a `LOG.debug` (or `LOG.warn`) in the error branch so 
the silent skip is observable, e.g.:
   ```java
   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.



-- 
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]

Reply via email to