You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/01/31 12:32:51 UTC

[hbase] branch branch-2.1 updated: HBASE-21684 Throw DNRIOE when connection or rpc client is closed

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

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


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new edbb816  HBASE-21684 Throw DNRIOE when connection or rpc client is closed
edbb816 is described below

commit edbb816abe2aa46b0adaabea1e2e0341880bccf3
Author: zhangduo <zh...@apache.org>
AuthorDate: Mon Jan 7 20:00:19 2019 +0800

    HBASE-21684 Throw DNRIOE when connection or rpc client is closed
---
 .../org/apache/hadoop/hbase/ipc/StoppedRpcClientException.java     | 4 ++--
 .../java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java   | 7 ++++---
 2 files changed, 6 insertions(+), 5 deletions(-)

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 d50004e..bd1e101 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 09f8984..3b50870 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.Delayed;
 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 @@ public final class ReadOnlyZKClient implements Closeable {
 
   public CompletableFuture<byte[]> get(String path) {
     if (closed.get()) {
-      return failed(new IOException("Client already closed"));
+      return failed(new DoNotRetryIOException("Client already closed"));
     }
     CompletableFuture<byte[]> future = new CompletableFuture<>();
     tasks.add(new ZKTask<byte[]>(path, future, "get") {
@@ -275,7 +276,7 @@ public final class ReadOnlyZKClient implements Closeable {
 
   public CompletableFuture<Stat> exists(String path) {
     if (closed.get()) {
-      return failed(new IOException("Client already closed"));
+      return failed(new DoNotRetryIOException("Client already closed"));
     }
     CompletableFuture<Stat> future = new CompletableFuture<>();
     tasks.add(new ZKTask<Stat>(path, future, "exists") {
@@ -339,7 +340,7 @@ public final class ReadOnlyZKClient implements Closeable {
       }
     }
     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();
   }