You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/11/18 04:16:22 UTC

[GitHub] [ratis] szetszwo opened a new pull request, #787: RATIS-1745. Improve performance of AwaitForSignal by reducing amount of exceptions created.

szetszwo opened a new pull request, #787:
URL: https://github.com/apache/ratis/pull/787

   See https://issues.apache.org/jira/browse/RATIS-1745


-- 
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: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] szetszwo commented on a diff in pull request #787: RATIS-1745. Improve performance of AwaitForSignal by reducing amount of exceptions created.

Posted by GitBox <gi...@apache.org>.
szetszwo commented on code in PR #787:
URL: https://github.com/apache/ratis/pull/787#discussion_r1026850067


##########
ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java:
##########
@@ -17,54 +17,54 @@
  */
 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.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();
 
   public AwaitForSignal(Object name) {
     this.name = name + "-" + JavaUtils.getClassSimpleName(getClass());
   }
 
   /** 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);
+      condition.await();
+    } finally {
+      lock.unlock();
     }
   }
 
   /** The same as {@link java.util.concurrent.locks.Condition#await(long, TimeUnit)} */
   public boolean await(long time, TimeUnit unit) throws InterruptedException {
-    if (time <= 0) {
-      return false;
-    }

Review Comment:
   I was thinking the the `condition.await(..)` will take care it.  However, it is a good idea to keep the if-statement as an optimization.  Let me add it back.  Thanks.



-- 
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: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] lhotari commented on a diff in pull request #787: RATIS-1745. Improve performance of AwaitForSignal by reducing amount of exceptions created.

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #787:
URL: https://github.com/apache/ratis/pull/787#discussion_r1026081437


##########
ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java:
##########
@@ -17,54 +17,54 @@
  */
 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.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();
 
   public AwaitForSignal(Object name) {
     this.name = name + "-" + JavaUtils.getClassSimpleName(getClass());
   }
 
   /** 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);
+      condition.await();
+    } finally {
+      lock.unlock();
     }
   }
 
   /** The same as {@link java.util.concurrent.locks.Condition#await(long, TimeUnit)} */
   public boolean await(long time, TimeUnit unit) throws InterruptedException {
-    if (time <= 0) {
-      return false;
-    }

Review Comment:
   Is it necessary to keep this 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: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] codings-dan merged pull request #787: RATIS-1745. Improve performance of AwaitForSignal by reducing amount of exceptions created.

Posted by GitBox <gi...@apache.org>.
codings-dan merged PR #787:
URL: https://github.com/apache/ratis/pull/787


-- 
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: issues-unsubscribe@ratis.apache.org

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


[GitHub] [ratis] lhotari commented on a diff in pull request #787: RATIS-1745. Improve performance of AwaitForSignal by reducing amount of exceptions created.

Posted by GitBox <gi...@apache.org>.
lhotari commented on code in PR #787:
URL: https://github.com/apache/ratis/pull/787#discussion_r1026081437


##########
ratis-common/src/main/java/org/apache/ratis/util/AwaitForSignal.java:
##########
@@ -17,54 +17,54 @@
  */
 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.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();
 
   public AwaitForSignal(Object name) {
     this.name = name + "-" + JavaUtils.getClassSimpleName(getClass());
   }
 
   /** 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);
+      condition.await();
+    } finally {
+      lock.unlock();
     }
   }
 
   /** The same as {@link java.util.concurrent.locks.Condition#await(long, TimeUnit)} */
   public boolean await(long time, TimeUnit unit) throws InterruptedException {
-    if (time <= 0) {
-      return false;
-    }

Review Comment:
   Would it be necessary to keep this 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: issues-unsubscribe@ratis.apache.org

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