Skip to content

Commit

Permalink
[HBASE-22591] : RecoverableZooKeeper improvement for getData, getChil… (
Browse files Browse the repository at this point in the history
#310)

HBASE-22591 RecoverableZooKeeper improvement for getData, getChildren, exists and removal of unused reference

Signed-off-by: Wellington Chevreuil <[email protected]>
  • Loading branch information
virajjasani authored and wchevreuil committed Jun 17, 2019
1 parent 5f2699e commit 214553d
Showing 1 changed file with 42 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@

import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.htrace.core.TraceScope;
Expand Down Expand Up @@ -169,7 +168,6 @@ public void delete(String path, int version) throws InterruptedException, Keeper
boolean isRetry = false; // False for first attempt, true for all retries.
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
checkZk().delete(path, version);
return;
} catch (KeeperException e) {
Expand Down Expand Up @@ -205,12 +203,21 @@ public void delete(String path, int version) throws InterruptedException, Keeper
* @return A Stat instance
*/
public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException {
return exists(path, watcher, null);
}

private Stat exists(String path, Watcher watcher, Boolean watch)
throws InterruptedException, KeeperException {
try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().exists(path, watcher);
Stat nodeStat;
if (watch == null) {
nodeStat = checkZk().exists(path, watcher);
} else {
nodeStat = checkZk().exists(path, watch);
}
return nodeStat;
} catch (KeeperException e) {
switch (e.code()) {
Expand All @@ -235,29 +242,7 @@ public Stat exists(String path, Watcher watcher) throws KeeperException, Interru
* @return A Stat instance
*/
public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException {
try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().exists(path, watch);
return nodeStat;
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
retryOrThrow(retryCounter, e, "exists");
break;
case OPERATIONTIMEOUT:
retryOrThrow(retryCounter, e, "exists");
break;

default:
throw e;
}
}
retryCounter.sleepUntilNextRetry();
}
}
return exists(path, null, watch);
}

private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
Expand All @@ -277,12 +262,21 @@ private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
*/
public List<String> getChildren(String path, Watcher watcher)
throws KeeperException, InterruptedException {
return getChildren(path, watcher, null);
}

private List<String> getChildren(String path, Watcher watcher, Boolean watch)
throws InterruptedException, KeeperException {
try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
List<String> children = checkZk().getChildren(path, watcher);
List<String> children;
if (watch == null) {
children = checkZk().getChildren(path, watcher);
} else {
children = checkZk().getChildren(path, watch);
}
return children;
} catch (KeeperException e) {
switch (e.code()) {
Expand All @@ -308,29 +302,7 @@ public List<String> getChildren(String path, Watcher watcher)
*/
public List<String> getChildren(String path, boolean watch)
throws KeeperException, InterruptedException {
try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
List<String> children = checkZk().getChildren(path, watch);
return children;
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
retryOrThrow(retryCounter, e, "getChildren");
break;
case OPERATIONTIMEOUT:
retryOrThrow(retryCounter, e, "getChildren");
break;

default:
throw e;
}
}
retryCounter.sleepUntilNextRetry();
}
}
return getChildren(path, null, watch);
}

/**
Expand All @@ -339,12 +311,21 @@ public List<String> getChildren(String path, boolean watch)
*/
public byte[] getData(String path, Watcher watcher, Stat stat)
throws KeeperException, InterruptedException {
return getData(path, watcher, null, stat);
}

private byte[] getData(String path, Watcher watcher, Boolean watch, Stat stat)
throws InterruptedException, KeeperException {
try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
byte[] revData = checkZk().getData(path, watcher, stat);
byte[] revData;
if (watch == null) {
revData = checkZk().getData(path, watcher, stat);
} else {
revData = checkZk().getData(path, watch, stat);
}
return ZKMetadata.removeMetaData(revData);
} catch (KeeperException e) {
switch (e.code()) {
Expand All @@ -370,29 +351,7 @@ public byte[] getData(String path, Watcher watcher, Stat stat)
*/
public byte[] getData(String path, boolean watch, Stat stat)
throws KeeperException, InterruptedException {
try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
byte[] revData = checkZk().getData(path, watch, stat);
return ZKMetadata.removeMetaData(revData);
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
retryOrThrow(retryCounter, e, "getData");
break;
case OPERATIONTIMEOUT:
retryOrThrow(retryCounter, e, "getData");
break;

default:
throw e;
}
}
retryCounter.sleepUntilNextRetry();
}
}
return getData(path, null, watch, stat);
}

/**
Expand All @@ -407,12 +366,9 @@ public Stat setData(String path, byte[] data, int version)
RetryCounter retryCounter = retryCounterFactory.create();
byte[] newData = ZKMetadata.appendMetaData(id, data);
boolean isRetry = false;
long startTime;
while (true) {
try {
startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().setData(path, newData, version);
return nodeStat;
return checkZk().setData(path, newData, version);
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
Expand Down Expand Up @@ -457,9 +413,7 @@ public List<ACL> getAcl(String path, Stat stat)
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
List<ACL> nodeACL = checkZk().getACL(path, stat);
return nodeACL;
return checkZk().getACL(path, stat);
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
Expand Down Expand Up @@ -488,9 +442,7 @@ public Stat setAcl(String path, List<ACL> acls, int version)
RetryCounter retryCounter = retryCounterFactory.create();
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
Stat nodeStat = checkZk().setACL(path, acls, version);
return nodeStat;
return checkZk().setACL(path, acls, version);
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
Expand Down Expand Up @@ -549,12 +501,9 @@ private String createNonSequential(String path, byte[] data, List<ACL> acl,
CreateMode createMode) throws KeeperException, InterruptedException {
RetryCounter retryCounter = retryCounterFactory.create();
boolean isRetry = false; // False for first attempt, true for all retries.
long startTime;
while (true) {
try {
startTime = EnvironmentEdgeManager.currentTime();
String nodePath = checkZk().create(path, data, acl, createMode);
return nodePath;
return checkZk().create(path, data, acl, createMode);
} catch (KeeperException e) {
switch (e.code()) {
case NODEEXISTS:
Expand Down Expand Up @@ -608,9 +557,7 @@ private String createSequential(String path, byte[] data,
}
}
first = false;
long startTime = EnvironmentEdgeManager.currentTime();
String nodePath = checkZk().create(newPath, data, acl, createMode);
return nodePath;
return checkZk().create(newPath, data, acl, createMode);
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
Expand Down Expand Up @@ -666,9 +613,7 @@ public List<OpResult> multi(Iterable<Op> ops)
Iterable<Op> multiOps = prepareZKMulti(ops);
while (true) {
try {
long startTime = EnvironmentEdgeManager.currentTime();
List<OpResult> opResults = checkZk().multi(multiOps);
return opResults;
return checkZk().multi(multiOps);
} catch (KeeperException e) {
switch (e.code()) {
case CONNECTIONLOSS:
Expand All @@ -693,12 +638,10 @@ private String findPreviousSequentialNode(String path)
assert(lastSlashIdx != -1);
String parent = path.substring(0, lastSlashIdx);
String nodePrefix = path.substring(lastSlashIdx+1);
long startTime = EnvironmentEdgeManager.currentTime();
List<String> nodes = checkZk().getChildren(parent, false);
List<String> matching = filterByPrefix(nodes, nodePrefix);
for (String node : matching) {
String nodePath = parent + "/" + node;
startTime = EnvironmentEdgeManager.currentTime();
Stat stat = checkZk().exists(nodePath, false);
if (stat != null) {
return nodePath;
Expand All @@ -725,10 +668,6 @@ public synchronized ZooKeeper getZooKeeper() {
return zk;
}

public synchronized byte[] getSessionPasswd() {
return zk == null ? null : zk.getSessionPasswd();
}

public void sync(String path, AsyncCallback.VoidCallback cb, Object ctx) throws KeeperException {
checkZk().sync(path, cb, null);
}
Expand Down

0 comments on commit 214553d

Please sign in to comment.