You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "ddanielr (via GitHub)" <gi...@apache.org> on 2023/03/09 00:12:10 UTC

[GitHub] [accumulo] ddanielr opened a new pull request, #3231: Fix wait timeout logic for available tservers

ddanielr opened a new pull request, #3231:
URL: https://github.com/apache/accumulo/pull/3231

   Replaces the retry object with a wait loop.
   Log messages are generated independent of max wait time.


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1132113127


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {
         log.info(
             "Blocking for tserver availability - need to reach {} servers. Have {}"
                 + " Time spent blocking {} sec.",
             minTserverCount, tserverSet.size(),
             TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
+        lastLog = System.currentTimeMillis();
+        logWait = logWait + logIncrement;
       }
     }
 
     if (tserverSet.size() < minTserverCount) {
       log.warn(
           "tserver availability check time expired - continuing. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
 
     } else {
       log.info(
           "tserver availability check completed. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
     }
+    sleepUninterruptibly(sleepInterval, TimeUnit.MILLISECONDS);

Review Comment:
   Switched back to using a retry object so this is OBE



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1132430540


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1489,40 +1490,41 @@ private void blockForTservers() throws InterruptedException {
           tserverSet.size(), Property.MASTER_STARTUP_TSERVER_AVAIL_MIN_COUNT.getKey());
       return;
     }
-    long maxWait = TimeUnit.MILLISECONDS
-        .toNanos(accConfig.getTimeInMillis(Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT));
+    long maxWait = accConfig.getTimeInMillis(Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT);
 
     if (maxWait <= 0) {
       log.info("tserver availability check set to block indefinitely, To change, set {} > 0.",
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
-    long sleepInterval = maxWait / 10;
 
-    // Set an incremental logging delay of 15 seconds.
-    long logIncrement = TimeUnit.SECONDS.toNanos(15);
-    long logWait = 0;
-    long lastLog = 0;
+    long retries = 10;
+    long waitPeriod = maxWait / retries;
+
+    Retry tserverRetry =
+        Retry.builder().maxRetries(retries).retryAfter(waitPeriod, TimeUnit.MILLISECONDS)
+            .incrementBy(0, TimeUnit.MILLISECONDS).maxWait(waitPeriod, TimeUnit.MILLISECONDS)
+            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();

Review Comment:
   just a nit - you can static import `TimeUnit.MILLISECONDS` so these are shorten to `MILLISECONDS`



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#issuecomment-1462171595

   The following seems close to what is wanted (I build this against 3.0, so there may be changes needed for 1.10)  Ignore the time and count values - they were picked for convenience and not what we'd what to use here.
   
   ```
        Retry.builder().maxRetries(10).retryAfter(50, MILLISECONDS).incrementBy(100, MILLISECONDS)
                           .maxWait(maxWaitSec, SECONDS).backOffFactor(2.0).logInterval(1, SECONDS).createRetry();
   
   ...
     while(retry.canRetry()){
               attempts++;
               retry.logRetry(log, "current counts " + attempts);
               retry.waitForNextAttempt(log, "pause for next attempt");
               retry.useRetry();
           }
   ```
   
   I think things to note:  
   
   - maxWait set the max delay and factors in with the backoff, the delay time will increase up to that value with each attempt.
   - maxRetries is a count 
   - removing infiniteRetries() and then adding useRetry is required to advance the retry attempt logic


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130229710


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;

Review Comment:
   When using TimeUnits, this may be clearer if it was logIncrementSec = 15 and then use the timeunit conversions. 



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130233046


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {
         log.info(
             "Blocking for tserver availability - need to reach {} servers. Have {}"
                 + " Time spent blocking {} sec.",
             minTserverCount, tserverSet.size(),
             TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
+        lastLog = System.currentTimeMillis();
+        logWait = logWait + logIncrement;
       }
     }
 
     if (tserverSet.size() < minTserverCount) {
       log.warn(
           "tserver availability check time expired - continuing. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
 
     } else {
       log.info(
           "tserver availability check completed. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
     }
+    sleepUninterruptibly(sleepInterval, TimeUnit.MILLISECONDS);

Review Comment:
   You should check the interrupt status.  This could be done with Thread.sleep() and a catch clause.  If `sleepUninterruptibly` is necessary because we *must* wait for the entire duration, the it should be noted with a comment that that behavior is necessary.



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130571197


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {
         log.info(
             "Blocking for tserver availability - need to reach {} servers. Have {}"
                 + " Time spent blocking {} sec.",
             minTserverCount, tserverSet.size(),
             TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
+        lastLog = System.currentTimeMillis();
+        logWait = logWait + logIncrement;
       }
     }
 
     if (tserverSet.size() < minTserverCount) {
       log.warn(
           "tserver availability check time expired - continuing. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));

Review Comment:
   Ended up static importing `NANOSECONDS` instead, due to the switch to`System.nanoTime()`



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#issuecomment-1461066428

   I don't think the `sleepInterval` is calculated elegantly. Dividing the wait interval into 10  sleep periods seemed to be ok for now, but I welcome better suggestions.
   
   I wanted to get this change in to close out the open items for 1.10.3
   


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130571979


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;

Review Comment:
   checkstyle didn't complain, but I switched it while I was in there. 



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130234161


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;

Review Comment:
   I think style conventions are that this should be two lines 
   ```
    long logWait = 0;
    long lastLog = 0;
   ```
   but not sure if checksytle will complain - if not the it can be left as is.



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #3231: Fix wait timeout logic for available tservers

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#issuecomment-1464584261

   GitHub's UI seems to create a new branch to revert the change if you click the revert button, even if you abort the process and don't actually follow through with reverting anything. I'll delete the unintentionally created revert branch. #3235 is a fix subsequent to this PR that should be finished before merging the 1.10 branch forward into 2.1 and on.


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#issuecomment-1461472973

   > Overall this looks okay, but removing the retry in favor of a wait loop looses some functionality - mainly that there was a small pause at start. It also seems cleaner because the wait times do not need to be calculated. Also, with the retry, there would be a most 3 log messages generated for the wait.
   > 
   > Why did you choose not to use retry?
   
   I attempted using a Retry with an earlier version of this fix. However, while wait times do not need to be calculated with a Retry, the amount of retries in the given time duration window still need to be calculated. Otherwise it never completes and just keeps retrying.  
   
   In my first iteration, the retry never matched up to my time value as the increment value would always cause the retry to overshoot the defined property value. 
   
   https://github.com/ddanielr/accumulo/pull/5/commits/f93c327073d660039716e21330deb98986b19494#diff-56945d7261689b2323a668699ab5865a1e48ec8424b0d612091d29ae0f2fc67cR1510
   
   Because of the complications with the Retry object, and the fact that`blockForTservers()` is not attempting an operation as it's just blocking, I chose to simplify it for a wait loop.
   
   I can add in a small wait at the start if you'd prefer, or I'm happy to dig further into the Retry object with someone and see if I'm just completely missing something.
   
   I'm not super concerned about the logs being spammed as this is a block on the main thread so nothing else should really be showing up in the logs until this method has completed.


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130231771


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {
         log.info(
             "Blocking for tserver availability - need to reach {} servers. Have {}"
                 + " Time spent blocking {} sec.",
             minTserverCount, tserverSet.size(),
             TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
+        lastLog = System.currentTimeMillis();
+        logWait = logWait + logIncrement;
       }
     }
 
     if (tserverSet.size() < minTserverCount) {
       log.warn(
           "tserver availability check time expired - continuing. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));

Review Comment:
   You can static import the time units so that it would be `MILLISECONDS` rather than `TimeUnit.MILLISECONDS`



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130570714


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {

Review Comment:
   Switched all the logic over to use `System.nanoTime()`



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman merged pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman merged PR #3231:
URL: https://github.com/apache/accumulo/pull/3231


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#issuecomment-1463488586

   Dug into the retry object a bit with @EdColeman and found that removing the incrementing backoff and just using maxRetries makes the Retry object function as expected. 
   
   I've pushed an updated commit that switches back to the Retry object, but calculates an exact max wait vs an approximate value. 
   It also switches the logging statement to use the Retry logging method which takes advantage of the Retry object's logging interval.
   


-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130570389


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;

Review Comment:
   I went a different direction with this due to the other requested changes for `System.nanoTime()`. 
   The variable is now set with an explicit unit of time `TimeUnit.SECONDS.toNanos(15);` which improves readability. 
   



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ddanielr commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "ddanielr (via GitHub)" <gi...@apache.org>.
ddanielr commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130571661


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {
         log.info(
             "Blocking for tserver availability - need to reach {} servers. Have {}"
                 + " Time spent blocking {} sec.",
             minTserverCount, tserverSet.size(),
             TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
+        lastLog = System.currentTimeMillis();
+        logWait = logWait + logIncrement;
       }
     }
 
     if (tserverSet.size() < minTserverCount) {
       log.warn(
           "tserver availability check time expired - continuing. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
 
     } else {
       log.info(
           "tserver availability check completed. Requested {}, have {} tservers on line. "
-              + " Time waiting {} ms",
+              + " Time waiting {} sec",
           tserverSet.size(), minTserverCount,
           TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - waitStart));
     }
+    sleepUninterruptibly(sleepInterval, TimeUnit.MILLISECONDS);

Review Comment:
   I had originally chosen an uninterruptible sleep call as there are 8 other instances of that method being called in this class without checking for interrupt status, but I'm indifferent on this and happy to switch it over to a normal sleep with interrupt. 
   
   Since `blockForTservers` is called from the main thread, do we need to handle the interrupt status via a try/catch block in the original `run` method? 
   
   Main creates a new master object and calls run
   https://github.com/apache/accumulo/blob/da2b7ed883a8e99c733fc557032b1a45544cddf1/server/master/src/main/java/org/apache/accumulo/master/Master.java#L1648-L1671
   
   Run calls `blockForTservers` in the same thread
   https://github.com/apache/accumulo/blob/da2b7ed883a8e99c733fc557032b1a45544cddf1/server/master/src/main/java/org/apache/accumulo/master/Master.java#L1258-L1275



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3231: Fix wait timeout logic for available tservers

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3231:
URL: https://github.com/apache/accumulo/pull/3231#discussion_r1130231191


##########
server/master/src/main/java/org/apache/accumulo/master/Master.java:
##########
@@ -1498,51 +1498,47 @@ private void blockForTservers() throws InterruptedException {
           Property.MASTER_STARTUP_TSERVER_AVAIL_MAX_WAIT.getKey());
       maxWait = Long.MAX_VALUE;
     }
+    long sleepInterval = maxWait / 10;
 
-    // honor Retry condition that initial wait < max wait, otherwise use small value to allow thread
-    // yield to happen
-    long initialWait = Math.min(50, maxWait / 2);
-
-    Retry tserverRetry =
-        Retry.builder().infiniteRetries().retryAfter(initialWait, TimeUnit.MILLISECONDS)
-            .incrementBy(15_000, TimeUnit.MILLISECONDS).maxWait(maxWait, TimeUnit.MILLISECONDS)
-            .logInterval(30_000, TimeUnit.MILLISECONDS).createRetry();
+    // Set a incremental logging delay
+    long logIncrement = 15_000;
+    long logWait = 0, lastLog = 0;
 
     log.info("Checking for tserver availability - need to reach {} servers. Have {}",
         minTserverCount, tserverSet.size());
 
     boolean needTservers = tserverSet.size() < minTserverCount;
 
-    while (needTservers && tserverRetry.canRetry()) {
-
-      tserverRetry.waitForNextAttempt();
-
+    while (needTservers && ((System.currentTimeMillis() - waitStart) < maxWait)) {
       needTservers = tserverSet.size() < minTserverCount;
 
-      // suppress last message once threshold reached.
-      if (needTservers) {
+      // Determine when to log a message
+      if (needTservers && ((System.currentTimeMillis() - lastLog) > logWait)) {

Review Comment:
   System.nanoTime() is preferred.  (Nanos are relative in the JVM, so you need to calculate the elapsed time with now - lastLog.)



-- 
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: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org