diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java index d50004eaf8db..bd1e101c2e95 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java @@ -17,11 +17,11 @@ */ package org.apache.hadoop.hbase.ipc; -import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Public -public class StoppedRpcClientException extends HBaseIOException { +public class StoppedRpcClientException extends DoNotRetryIOException { public StoppedRpcClientException() { super(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java index 09f8984abf39..3b508707f2ce 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java @@ -30,6 +30,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -259,7 +260,7 @@ private static CompletableFuture failed(Throwable e) { public CompletableFuture get(String path) { if (closed.get()) { - return failed(new IOException("Client already closed")); + return failed(new DoNotRetryIOException("Client already closed")); } CompletableFuture future = new CompletableFuture<>(); tasks.add(new ZKTask(path, future, "get") { @@ -275,7 +276,7 @@ protected void doExec(ZooKeeper zk) { public CompletableFuture exists(String path) { if (closed.get()) { - return failed(new IOException("Client already closed")); + return failed(new DoNotRetryIOException("Client already closed")); } CompletableFuture future = new CompletableFuture<>(); tasks.add(new ZKTask(path, future, "exists") { @@ -339,7 +340,7 @@ private void run() { } } closeZk(); - IOException error = new IOException("Client already closed"); + DoNotRetryIOException error = new DoNotRetryIOException("Client already closed"); Arrays.stream(tasks.toArray(new Task[0])).forEach(t -> t.closed(error)); tasks.clear(); }