This is an automated email from the ASF dual-hosted git repository.

shashikant 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 b98ff40  RATIS-1414. In TaskQueue, a task future should be completed 
only after the task is polled. (#517)
b98ff40 is described below

commit b98ff4065811b0d3aeaa747b4d05df958ee8ef07
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Fri Oct 8 17:08:44 2021 +0800

    RATIS-1414. In TaskQueue, a task future should be completed only after the 
task is polled. (#517)
---
 .../main/java/org/apache/ratis/util/TaskQueue.java | 16 ++++++-
 .../java/org/apache/ratis/util/TestTaskQueue.java  | 49 ++++++++++++++++++++++
 2 files changed, 63 insertions(+), 2 deletions(-)

diff --git a/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java 
b/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
index 9d7350c..858739a 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/TaskQueue.java
@@ -113,13 +113,25 @@ public class TaskQueue {
     final CompletableFuture<OUTPUT> f = new CompletableFuture<>();
     final Runnable runnable = LogUtils.newRunnable(LOG, () -> {
       LOG.trace("{}: running {}", this, task);
+
+      // run the task and wait for it to complete
+      OUTPUT output = null;
+      Throwable throwable = null;
       try {
-        f.complete(task.get());
+        output = task.get();
       } catch (Throwable t) {
-        f.completeExceptionally(newThrowable.apply(t));
+        throwable = t;
       }
 
+      // poll the task and then submit the next task
       pollAndSubmit(executor);
+
+      // complete the future after poll
+      if (throwable != null) {
+        f.completeExceptionally(newThrowable.apply(throwable));
+      } else {
+        f.complete(output);
+      }
     }, task::toString);
 
     offerAndSubmit(runnable, executor);
diff --git 
a/ratis-common/src/test/java/org/apache/ratis/util/TestTaskQueue.java 
b/ratis-common/src/test/java/org/apache/ratis/util/TestTaskQueue.java
new file mode 100644
index 0000000..b0e5870
--- /dev/null
+++ b/ratis-common/src/test/java/org/apache/ratis/util/TestTaskQueue.java
@@ -0,0 +1,49 @@
+/*
+ * 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
+ *
+ *     http://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.ratis.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadLocalRandom;
+
+public class TestTaskQueue {
+  static int randomSleep(int id) {
+    final int sleepMs = 10 + ThreadLocalRandom.current().nextInt(10);
+    try {
+      Thread.sleep(sleepMs);
+    } catch (InterruptedException e) {
+      throw new IllegalStateException(e);
+    }
+    return id;
+  }
+
+  @Test
+  public void testIsEmpty() throws Exception {
+    final ExecutorService executor = Executors.newSingleThreadExecutor();
+    final TaskQueue q = new TaskQueue("testing");
+    for(int i = 0; i < 10; i++) {
+      final int id = i;
+      q.submit(() -> randomSleep(id), executor)
+          .thenAccept(j -> Assert.assertTrue("Queue is not empty after task " 
+ id + " completed", q.isEmpty()))
+          .get();
+    }
+  }
+}

Reply via email to