This is an automated email from the ASF dual-hosted git repository.
dragonyliu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new 9d1e98956 RATIS-1745. Improve performance of AwaitForSignal by
reducing amount of exceptions created. (#787)
9d1e98956 is described below
commit 9d1e9895666936d911758d04b4ea3f8386694abe
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Tue Nov 29 04:20:15 2022 -0800
RATIS-1745. Improve performance of AwaitForSignal by reducing amount of
exceptions created. (#787)
---
.../java/org/apache/ratis/util/AwaitForSignal.java | 41 +++++++++++++---------
1 file changed, 25 insertions(+), 16 deletions(-)
diff --git
a/ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java
b/ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java
index 5294818f2..88808c52b 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java
@@ -17,22 +17,24 @@
*/
package org.apache.ratis.util;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
/**
* This class is a partial implementation of {@link
java.util.concurrent.locks.Condition}.
* Only some of the await and signal methods are implemented.
- *
+ * <p>
* This class is threadsafe.
*/
public class AwaitForSignal {
private final String name;
-
- private final AtomicReference<CompletableFuture<Void>> future = new
AtomicReference<>(new CompletableFuture<>());
+ private final Lock lock = new ReentrantLock();
+ private final Condition condition = lock.newCondition();
+ private final AtomicReference<AtomicBoolean> signaled = new
AtomicReference<>(new AtomicBoolean());
public AwaitForSignal(Object name) {
this.name = name + "-" + JavaUtils.getClassSimpleName(getClass());
@@ -40,10 +42,13 @@ public class AwaitForSignal {
/** The same as {@link java.util.concurrent.locks.Condition#await()} */
public void await() throws InterruptedException {
+ lock.lock();
try {
- future.get().get();
- } catch (ExecutionException e) {
- throw new IllegalStateException(e);
+ for (final AtomicBoolean s = signaled.get(); !s.get(); ) {
+ condition.await();
+ }
+ } finally {
+ lock.unlock();
}
}
@@ -52,19 +57,23 @@ public class AwaitForSignal {
if (time <= 0) {
return false;
}
+ lock.lock();
try {
- future.get().get(time, unit);
- } catch (ExecutionException e) {
- throw new IllegalStateException(e);
- } catch (TimeoutException ignored) {
- return false;
+ return condition.await(time, unit);
+ } finally {
+ lock.unlock();
}
- return true;
}
/** The same as {@link java.util.concurrent.locks.Condition#signal()} */
public void signal() {
- future.getAndSet(new CompletableFuture<>()).complete(null);
+ lock.lock();
+ try {
+ signaled.getAndSet(new AtomicBoolean()).set(true);
+ condition.signalAll();
+ } finally {
+ lock.unlock();
+ }
}
@Override