You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "EdColeman (via GitHub)" <gi...@apache.org> on 2023/11/30 20:45:55 UTC

[PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

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

   The test module has a convenient Wait.waitFor that could be used generally, however, that code allows for a scaling factor to allow for variations in test environments.  This PR replicates that functionality for general use outside of the test module.
   
   Rather that static methods, this uses fluent-style builder that can be concise while providing customization. It also has a signature different from the test waitFor methods to avoid confusion which wait is being used.
   
   This also relocates the sleep method from UtilWaitThread and deletes that class.  The original intent of UtilWaitThread class was to hold guava sleepUninterruptibly code that had been marked as beta at one time.  Using the guava provided sleepUninterruptibly was completed in a previous PR.  The relocated sleep also reestablishes the interrupt flag that can be used by callers to determine if in interrupt occurred. 


-- 
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


Re: [PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

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

   It is currently unused because I was waiting for some version to get approved so that I could move forward.  There are other PRs that I have open where it was suggested that I use Wait.waitFor - which seemed like an improvement, so I started down this path instead.  
   
   I didn't think that the other PR changes needed to be bundled into this issue.


-- 
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


Re: [PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

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


##########
core/src/main/java/org/apache/accumulo/core/util/WaitFor.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a generic capability to sleep until a condition is true. The maximum time to wait for
+ * the condition, the delay between checking the condition as well as progress and error messages
+ * can be provided. All are optional.
+ * <p>
+ * If the timeout is reached, an interrupt occurs, or if an Exception is thrown checking the
+ * condition an IllegalStateException is thrown.
+ * <p>
+ * Note: Integration tests should use the @{code Wait.waitFor(...) methods in the test module that
+ * allow the delays to be scaled using a timeout factor to allow for variations in test
+ * environments.}
+ */
+public class WaitFor {
+  private static final Logger LOG = LoggerFactory.getLogger(WaitFor.class);
+  private static final long MAX_DURATION_SEC = 30;
+  private static final long MAX_SLEEP_SEC = 1;
+
+  /**
+   * Convenience method that sleeps for the specified time (in milliseconds) where callers do not
+   * need to catch / handle the InterruptedException. If an interrupt occurs, the interrupt is
+   * reasserted so the caller has the option to test if an interrupt occurred and can take
+   * appropriate action.
+   * <p>
+   * Using this method should be discouraged in favor of the caller properly catching and taking
+   * appropriate action. At a minimum, callers should test for the interrupt status on return and
+   * take action if an interrupt occurred.
+   *
+   * @param millis the sleep time.
+   */
+  public static void sleep(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("{}", e.getMessage(), e);
+    }
+  }
+
+  public interface Condition {
+    boolean isSatisfied() throws Exception;
+  }
+
+  private long duration = SECONDS.toMillis(MAX_DURATION_SEC);
+  private long sleepMillis = SECONDS.toMillis(MAX_SLEEP_SEC);
+
+  private String progressMsg = "";
+  private String failMessage = "";
+
+  private Condition condition = null;
+
+  /**
+   * Use a {@link #builder(Condition)} to create an instance;
+   */
+  private WaitFor() {}
+
+  /**
+   * Set the condition required to be true before the {@link #waitFor()} will continue, This creates
+   * a fluent-style object to allow for setting optional parameters before calling
+   * {@link #waitFor()}.
+   *
+   * @param condition when condition evaluates true, return from waiting
+   * @return return a fluent-style object to configure optional parameters.
+   */
+  public static WaitFor builder(@NonNull final Condition condition) {
+    WaitFor waiter = new WaitFor();
+    waiter.condition = condition;
+    return waiter;
+  }
+
+  /**
+   * Set the approximate maximum time the waitFor will wait for the condition to be satisfied. If
+   * the condition is not satisfied by this time, an IllegalStateException will be thrown. The
+   * absolute max time for the wait also depends on the delay time and could be up to (duration +
+   * delay). If not set, the default value is {@value #MAX_DURATION_SEC} seconds.
+   *
+   * @param duration the approximate max wait time for the condition to be true.
+   * @param units the time units of the duration.
+   * @return return a fluent-style object to configure other optional parameters.
+   */
+  public WaitFor upTo(final long duration, TimeUnit units) {
+    if (duration <= 0) {

Review Comment:
   Rather than fail I opted to fall back to the default value. 



-- 
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


Re: [PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman closed pull request #4002: Add waitFor method in core. Relocate UtilWaitThread.sleep
URL: https://github.com/apache/accumulo/pull/4002


-- 
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


Re: [PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

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


##########
core/src/main/java/org/apache/accumulo/core/util/WaitFor.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Provides a generic capability to sleep until a condition is true. The maximum time to wait for
+ * the condition, the delay between checking the condition as well as progress and error messages
+ * can be provided. All are optional.
+ * <p>
+ * If the timeout is reached, an interrupt occurs, or if an Exception is thrown checking the
+ * condition an IllegalStateException is thrown.
+ * <p>
+ * Note: Integration tests should use the @{code Wait.waitFor(...) methods in the test module that
+ * allow the delays to be scaled using a timeout factor to allow for variations in test
+ * environments.}
+ */
+public class WaitFor {
+  private static final Logger LOG = LoggerFactory.getLogger(WaitFor.class);
+  private static final long MAX_DURATION_SEC = 30;
+  private static final long MAX_SLEEP_SEC = 1;
+
+  /**
+   * Convenience method that sleeps for the specified time (in milliseconds) where callers do not
+   * need to catch / handle the InterruptedException. If an interrupt occurs, the interrupt is
+   * reasserted so the caller has the option to test if an interrupt occurred and can take
+   * appropriate action.
+   * <p>
+   * Using this method should be discouraged in favor of the caller properly catching and taking
+   * appropriate action. At a minimum, callers should test for the interrupt status on return and
+   * take action if an interrupt occurred.
+   *
+   * @param millis the sleep time.
+   */
+  public static void sleep(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("{}", e.getMessage(), e);

Review Comment:
   could improve this message a bit while we're here
   
   ```suggestion
         LOG.error("sleep interrupted", e);
   ```



-- 
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


Re: [PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

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


##########
core/src/main/java/org/apache/accumulo/core/util/WaitFor.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a generic capability to sleep until a condition is true. The maximum time to wait for
+ * the condition, the delay between checking the condition as well as progress and error messages
+ * can be provided. All are optional.
+ * <p>
+ * If the timeout is reached, an interrupt occurs, or if an Exception is thrown checking the
+ * condition an IllegalStateException is thrown.
+ * <p>
+ * Note: Integration tests should use the @{code Wait.waitFor(...) methods in the test module that
+ * allow the delays to be scaled using a timeout factor to allow for variations in test
+ * environments.}
+ */
+public class WaitFor {
+  private static final Logger LOG = LoggerFactory.getLogger(WaitFor.class);
+  private static final long MAX_DURATION_SEC = 30;
+  private static final long MAX_SLEEP_SEC = 1;
+
+  /**
+   * Convenience method that sleeps for the specified time (in milliseconds) where callers do not
+   * need to catch / handle the InterruptedException. If an interrupt occurs, the interrupt is
+   * reasserted so the caller has the option to test if an interrupt occurred and can take
+   * appropriate action.
+   * <p>
+   * Using this method should be discouraged in favor of the caller properly catching and taking
+   * appropriate action. At a minimum, callers should test for the interrupt status on return and
+   * take action if an interrupt occurred.
+   *
+   * @param millis the sleep time.
+   */
+  public static void sleep(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("{}", e.getMessage(), e);
+    }
+  }
+
+  public interface Condition {
+    boolean isSatisfied() throws Exception;
+  }
+
+  private long duration = SECONDS.toMillis(MAX_DURATION_SEC);
+  private long sleepMillis = SECONDS.toMillis(MAX_SLEEP_SEC);
+
+  private String progressMsg = "";
+  private String failMessage = "";
+
+  private Condition condition = null;
+
+  /**
+   * Use a {@link #builder(Condition)} to create an instance;
+   */
+  private WaitFor() {}
+
+  /**
+   * Set the condition required to be true before the {@link #waitFor()} will continue, This creates
+   * a fluent-style object to allow for setting optional parameters before calling
+   * {@link #waitFor()}.
+   *
+   * @param condition when condition evaluates true, return from waiting
+   * @return return a fluent-style object to configure optional parameters.
+   */
+  public static WaitFor builder(@NonNull final Condition condition) {
+    WaitFor waiter = new WaitFor();
+    waiter.condition = condition;
+    return waiter;
+  }
+
+  /**
+   * Set the approximate maximum time the waitFor will wait for the condition to be satisfied. If
+   * the condition is not satisfied by this time, an IllegalStateException will be thrown. The
+   * absolute max time for the wait also depends on the delay time and could be up to (duration +
+   * delay). If not set, the default value is {@value #MAX_DURATION_SEC} seconds.
+   *
+   * @param duration the approximate max wait time for the condition to be true.
+   * @param units the time units of the duration.
+   * @return return a fluent-style object to configure other optional parameters.
+   */
+  public WaitFor upTo(final long duration, TimeUnit units) {
+    if (duration <= 0) {

Review Comment:
   Why not use `Preconditions.checkArgument()` if the inputs are invalid?



-- 
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


Re: [PR] Add waitFor method in core. Relocate UtilWaitThread.sleep [accumulo]

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


##########
core/src/main/java/org/apache/accumulo/core/util/WaitFor.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.core.util;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+import java.util.concurrent.TimeUnit;
+
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides a generic capability to sleep until a condition is true. The maximum time to wait for
+ * the condition, the delay between checking the condition as well as progress and error messages
+ * can be provided. All are optional.
+ * <p>
+ * If the timeout is reached, an interrupt occurs, or if an Exception is thrown checking the
+ * condition an IllegalStateException is thrown.
+ * <p>
+ * Note: Integration tests should use the @{code Wait.waitFor(...) methods in the test module that
+ * allow the delays to be scaled using a timeout factor to allow for variations in test
+ * environments.}
+ */
+public class WaitFor {
+  private static final Logger LOG = LoggerFactory.getLogger(WaitFor.class);
+  private static final long MAX_DURATION_SEC = 30;
+  private static final long MAX_SLEEP_SEC = 1;
+
+  /**
+   * Convenience method that sleeps for the specified time (in milliseconds) where callers do not
+   * need to catch / handle the InterruptedException. If an interrupt occurs, the interrupt is
+   * reasserted so the caller has the option to test if an interrupt occurred and can take
+   * appropriate action.
+   * <p>
+   * Using this method should be discouraged in favor of the caller properly catching and taking
+   * appropriate action. At a minimum, callers should test for the interrupt status on return and
+   * take action if an interrupt occurred.
+   *
+   * @param millis the sleep time.
+   */
+  public static void sleep(long millis) {
+    try {
+      Thread.sleep(millis);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      LOG.error("{}", e.getMessage(), e);
+    }
+  }
+
+  public interface Condition {
+    boolean isSatisfied() throws Exception;
+  }
+
+  private long duration = SECONDS.toMillis(MAX_DURATION_SEC);
+  private long sleepMillis = SECONDS.toMillis(MAX_SLEEP_SEC);
+
+  private String progressMsg = "";
+  private String failMessage = "";
+
+  private Condition condition = null;
+
+  /**
+   * Use a {@link #builder(Condition)} to create an instance;
+   */
+  private WaitFor() {}
+
+  /**
+   * Set the condition required to be true before the {@link #waitFor()} will continue, This creates
+   * a fluent-style object to allow for setting optional parameters before calling
+   * {@link #waitFor()}.
+   *
+   * @param condition when condition evaluates true, return from waiting
+   * @return return a fluent-style object to configure optional parameters.
+   */
+  public static WaitFor builder(@NonNull final Condition condition) {
+    WaitFor waiter = new WaitFor();
+    waiter.condition = condition;
+    return waiter;
+  }
+
+  /**
+   * Set the approximate maximum time the waitFor will wait for the condition to be satisfied. If
+   * the condition is not satisfied by this time, an IllegalStateException will be thrown. The
+   * absolute max time for the wait also depends on the delay time and could be up to (duration +
+   * delay). If not set, the default value is {@value #MAX_DURATION_SEC} seconds.
+   *
+   * @param duration the approximate max wait time for the condition to be true.
+   * @param units the time units of the duration.
+   * @return return a fluent-style object to configure other optional parameters.
+   */
+  public WaitFor upTo(final long duration, TimeUnit units) {
+    if (duration <= 0) {

Review Comment:
   Changed to use `Preconditions.checkArgument()` in 775897dfc8



-- 
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