HBASE-21684 Throw DNRIOE when connection or rpc client is closed
This commit is contained in:
parent
70fb6951d8
commit
e61c12c46f
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue