This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch branch-3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-3 by this push:
     new b71c125a45f HBASE-29817 Stop retrying on DoNotRetryIOException when 
waiting procedure result (#7610)
b71c125a45f is described below

commit b71c125a45f7b5775f8ed1d7c5832631d978d2ef
Author: Duo Zhang <[email protected]>
AuthorDate: Fri Jan 9 17:37:21 2026 +0800

    HBASE-29817 Stop retrying on DoNotRetryIOException when waiting procedure 
result (#7610)
    
    Signed-off-by: Hui Ruan <[email protected]>
    Reviewed-by: Umesh Kumar <[email protected]>
    Reviewed-by: Liu Xiao <[email protected]>
    (cherry picked from commit 9e3b53cfcb13552726f57dab45ce337bef5c8032)
---
 .../org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java  | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)

diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 381e1449975..afb611f2368 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -3119,10 +3119,15 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
         .call(),
       (response, error) -> {
         if (error != null) {
-          LOG.warn("failed to get the procedure result procId={}", procId,
-            ConnectionUtils.translateException(error));
-          retryTimer.newTimeout(t -> getProcedureResult(procId, converter, 
future, retries + 1),
-            ConnectionUtils.getPauseTime(pauseNs, retries), 
TimeUnit.NANOSECONDS);
+          Throwable exc = ConnectionUtils.translateException(error);
+          if (exc instanceof DoNotRetryIOException) {
+            // stop retrying on DNRIOE
+            future.completeExceptionally(exc);
+          } else {
+            LOG.warn("failed to get the procedure result procId={}", procId, 
exc);
+            retryTimer.newTimeout(t -> getProcedureResult(procId, converter, 
future, retries + 1),
+              ConnectionUtils.getPauseTime(pauseNs, retries), 
TimeUnit.NANOSECONDS);
+          }
           return;
         }
         if (response.getState() == GetProcedureResultResponse.State.RUNNING) {

Reply via email to