virajjasani commented on code in PR #5837:
URL: https://github.com/apache/hbase/pull/5837#discussion_r1633777863


##########
hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java:
##########
@@ -289,6 +346,28 @@ protected void doExec(ZooKeeper zk) {
     return future;
   }
 
+  public CompletableFuture<List<String>> listWithTimeout(String path, long 
endTime) {
+    CompletableFuture<List<String>> future = list(path);
+    TimerTask timerTask = new TimerTask() {
+      @Override
+      public void run(Timeout timeout) throws Exception {
+        if (EnvironmentEdgeManager.currentTime() > endTime) {
+          if (!future.isCancelled() && !future.isDone() && 
!future.isCompletedExceptionally()) {
+            future.completeExceptionally(
+              new DoNotRetryIOException("Zookeeper get could not be completed 
by " + endTime));
+          }
+        } else {
+          if (!future.isCancelled() && !future.isDone() && 
!future.isCompletedExceptionally()) {
+            RETRY_TIMER.newTimeout(this, 10, TimeUnit.MILLISECONDS);

Review Comment:
   same here, let's keep it 1ms



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java:
##########
@@ -76,8 +83,14 @@ public final class ReadOnlyZKClient implements Closeable {
 
   private final int keepAliveTimeMs;
 
+  public static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer(
+    new 
ThreadFactoryBuilder().setNameFormat("Async-Client-Retry-Timer-pool-%d").setDaemon(true)

Review Comment:
   The name format should be `Read-Only-ZKClient-Retry-Timer-pool-%d`



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java:
##########
@@ -274,6 +309,28 @@ protected void doExec(ZooKeeper zk) {
     return future;
   }
 
+  public CompletableFuture<Stat> existsWithTimeout(String path, int endTime) {
+    CompletableFuture<Stat> future = exists(path);
+    TimerTask timerTask = new TimerTask() {
+      @Override
+      public void run(Timeout timeout) throws Exception {
+        if (EnvironmentEdgeManager.currentTime() > endTime) {
+          if (!future.isCancelled() && !future.isDone() && 
!future.isCompletedExceptionally()) {
+            future.completeExceptionally(
+              new DoNotRetryIOException("Zookeeper get could not be completed 
by " + endTime));
+          }
+        } else {
+          if (!future.isCancelled() && !future.isDone() && 
!future.isCompletedExceptionally()) {
+            RETRY_TIMER.newTimeout(this, 10, TimeUnit.MILLISECONDS);

Review Comment:
   let's keep 1, 10 was just the example i posted



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to