HBASE-19772 ReadOnlyZKClient improvements
This commit is contained in:
parent
a7f9668c33
commit
70515f5311
|
@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
@ -116,6 +117,8 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
|
|
||||||
private ZooKeeper zookeeper;
|
private ZooKeeper zookeeper;
|
||||||
|
|
||||||
|
private int pendingRequests = 0;
|
||||||
|
|
||||||
private String getId() {
|
private String getId() {
|
||||||
return String.format("0x%08x", System.identityHashCode(this));
|
return String.format("0x%08x", System.identityHashCode(this));
|
||||||
}
|
}
|
||||||
|
@ -127,12 +130,12 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
this.retryIntervalMs =
|
this.retryIntervalMs =
|
||||||
conf.getInt(RECOVERY_RETRY_INTERVAL_MILLIS, DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS);
|
conf.getInt(RECOVERY_RETRY_INTERVAL_MILLIS, DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS);
|
||||||
this.keepAliveTimeMs = conf.getInt(KEEPALIVE_MILLIS, DEFAULT_KEEPALIVE_MILLIS);
|
this.keepAliveTimeMs = conf.getInt(KEEPALIVE_MILLIS, DEFAULT_KEEPALIVE_MILLIS);
|
||||||
LOG.info("Start read only zookeeper connection " + getId() + " to " + connectString +
|
LOG.info(
|
||||||
", session timeout " + sessionTimeoutMs + " ms, retries " + maxRetries +
|
"Start read only zookeeper connection {} to {}, " + "session timeout {} ms, retries {}, " +
|
||||||
", retry interval " + retryIntervalMs + " ms, keep alive " + keepAliveTimeMs + " ms");
|
"retry interval {} ms, keep alive {} ms",
|
||||||
Thread t = new Thread(this::run, "ReadOnlyZKClient");
|
getId(), connectString, sessionTimeoutMs, maxRetries, retryIntervalMs, keepAliveTimeMs);
|
||||||
t.setDaemon(true);
|
Threads.setDaemonThreadRunning(new Thread(this::run),
|
||||||
t.start();
|
"ReadOnlyZKClient-" + connectString + "@" + getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
private abstract class ZKTask<T> extends Task {
|
private abstract class ZKTask<T> extends Task {
|
||||||
|
@ -156,6 +159,7 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void exec(ZooKeeper alwaysNull) {
|
public void exec(ZooKeeper alwaysNull) {
|
||||||
|
pendingRequests--;
|
||||||
Code code = Code.get(rc);
|
Code code = Code.get(rc);
|
||||||
if (code == Code.OK) {
|
if (code == Code.OK) {
|
||||||
future.complete(ret);
|
future.complete(ret);
|
||||||
|
@ -169,19 +173,19 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
future.completeExceptionally(KeeperException.create(code, path));
|
future.completeExceptionally(KeeperException.create(code, path));
|
||||||
} else {
|
} else {
|
||||||
if (code == Code.SESSIONEXPIRED) {
|
if (code == Code.SESSIONEXPIRED) {
|
||||||
LOG.warn(getId() + " session expired, close and reconnect");
|
LOG.warn("{} to {} session expired, close and reconnect", getId(), connectString);
|
||||||
try {
|
try {
|
||||||
zk.close();
|
zk.close();
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (ZKTask.this.delay(retryIntervalMs, maxRetries)) {
|
if (ZKTask.this.delay(retryIntervalMs, maxRetries)) {
|
||||||
LOG.warn(getId() + " failed for " + operationType + " of " + path + ", code = " +
|
LOG.warn("{} to {} failed for {} of {}, code = {}, retries = {}", getId(),
|
||||||
code + ", retries = " + ZKTask.this.retries);
|
connectString, operationType, path, code, ZKTask.this.retries);
|
||||||
tasks.add(ZKTask.this);
|
tasks.add(ZKTask.this);
|
||||||
} else {
|
} else {
|
||||||
LOG.warn(getId() + " failed for " + operationType + " of " + path + ", code = " +
|
LOG.warn("{} to {} failed for {} of {}, code = {}, retries = {}, give up", getId(),
|
||||||
code + ", retries = " + ZKTask.this.retries + ", give up");
|
connectString, operationType, path, code, ZKTask.this.retries);
|
||||||
future.completeExceptionally(KeeperException.create(code, path));
|
future.completeExceptionally(KeeperException.create(code, path));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -205,6 +209,14 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected abstract void doExec(ZooKeeper zk);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final void exec(ZooKeeper zk) {
|
||||||
|
pendingRequests++;
|
||||||
|
doExec(zk);
|
||||||
|
}
|
||||||
|
|
||||||
public boolean delay(long intervalMs, int maxRetries) {
|
public boolean delay(long intervalMs, int maxRetries) {
|
||||||
if (retries >= maxRetries) {
|
if (retries >= maxRetries) {
|
||||||
return false;
|
return false;
|
||||||
|
@ -217,14 +229,12 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
@Override
|
@Override
|
||||||
public void connectFailed(IOException e) {
|
public void connectFailed(IOException e) {
|
||||||
if (delay(retryIntervalMs, maxRetries)) {
|
if (delay(retryIntervalMs, maxRetries)) {
|
||||||
LOG.warn(getId() + " failed to connect to zk for " + operationType + " of " + path +
|
LOG.warn("{} to {} failed to connect to zk fo {} of {}, retries = {}", getId(),
|
||||||
", retries = " + retries,
|
connectString, operationType, path, retries, e);
|
||||||
e);
|
|
||||||
tasks.add(this);
|
tasks.add(this);
|
||||||
} else {
|
} else {
|
||||||
LOG.warn(getId() + " failed to connect to zk for " + operationType + " of " + path +
|
LOG.warn("{} to {} failed to connect to zk fo {} of {}, retries = {}, give up", getId(),
|
||||||
", retries = " + retries + ", give up",
|
connectString, operationType, path, retries, e);
|
||||||
e);
|
|
||||||
future.completeExceptionally(e);
|
future.completeExceptionally(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -249,7 +259,7 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
tasks.add(new ZKTask<byte[]>(path, future, "get") {
|
tasks.add(new ZKTask<byte[]>(path, future, "get") {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void exec(ZooKeeper zk) {
|
protected void doExec(ZooKeeper zk) {
|
||||||
zk.getData(path, false,
|
zk.getData(path, false,
|
||||||
(rc, path, ctx, data, stat) -> onComplete(zk, rc, data, true), null);
|
(rc, path, ctx, data, stat) -> onComplete(zk, rc, data, true), null);
|
||||||
}
|
}
|
||||||
|
@ -265,7 +275,7 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
tasks.add(new ZKTask<Stat>(path, future, "exists") {
|
tasks.add(new ZKTask<Stat>(path, future, "exists") {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void exec(ZooKeeper zk) {
|
protected void doExec(ZooKeeper zk) {
|
||||||
zk.exists(path, false, (rc, path, ctx, stat) -> onComplete(zk, rc, stat, false), null);
|
zk.exists(path, false, (rc, path, ctx, stat) -> onComplete(zk, rc, stat, false), null);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -311,9 +321,11 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
if (task == CLOSE) {
|
if (task == CLOSE) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
if (task == null) {
|
if (task == null && pendingRequests == 0) {
|
||||||
LOG.info(getId() + " no activities for " + keepAliveTimeMs +
|
LOG.info(
|
||||||
" ms, close active connection. Will reconnect next time when there are new requests.");
|
"{} to {} no activities for {} ms, close active connection. " +
|
||||||
|
"Will reconnect next time when there are new requests",
|
||||||
|
getId(), connectString, keepAliveTimeMs);
|
||||||
closeZk();
|
closeZk();
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
@ -339,7 +351,7 @@ public final class ReadOnlyZKClient implements Closeable {
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
if (closed.compareAndSet(false, true)) {
|
if (closed.compareAndSet(false, true)) {
|
||||||
LOG.info("Close zookeeper connection " + getId() + " to " + connectString);
|
LOG.info("Close zookeeper connection {} to {}", getId(), connectString);
|
||||||
tasks.add(CLOSE);
|
tasks.add(CLOSE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue