>From Ali Alsuliman <[email protected]>:

Ali Alsuliman has uploaded this change for review. ( 
https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19466 )


Change subject: [NO ISSUE][HYR] Retry cancelling tasks only after the timeout
......................................................................

[NO ISSUE][HYR] Retry cancelling tasks only after the timeout

- user model changes: no
- storage format changes: no
- interface changes: no

Instead of retrying the tasks cancellation on both
the 5 minutes timeout or interrupt, retry only after the timeout
to avoid retrying with every interrupt that can happen to
the Super Activity.

Ext-ref: MB-65432

Change-Id: Ie585127fe30904f5126bae8867b94ea12cd45762
---
M 
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
1 file changed, 41 insertions(+), 17 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb 
refs/changes/66/19466/1

diff --git 
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
 
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index e031284..a4dd9db 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -52,6 +52,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.api.util.InvokeUtil;
+import org.apache.hyracks.util.Span;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;

@@ -282,25 +284,27 @@
     }

     private static boolean cancelTasks(List<Future<Void>> tasks, Semaphore 
completeSemaphore) {
-        boolean interrupted = Thread.interrupted();
-        try {
-            while (true) {
-                for (Future<Void> task : tasks) {
-                    task.cancel(true);
-                }
-                try {
-                    if (completeSemaphore.tryAcquire(5, TimeUnit.MINUTES)) {
-                        return true;
-                    }
-                    LOGGER.warn("not all tasks were cancelled within 5 
minutes. retrying cancelling...");
-                } catch (InterruptedException e) {
-                    interrupted = true;
-                }
+        Span retryWait = Span.init(5, TimeUnit.MINUTES);
+        while (true) {
+            for (Future<Void> task : tasks) {
+                task.cancel(true);
             }
-        } finally {
-            if (interrupted) {
-                Thread.currentThread().interrupt();
+            if (acquireUninterruptibly(completeSemaphore, retryWait)) {
+                return true;
             }
+            LOGGER.warn("not all tasks were cancelled within 5 minutes. 
retrying cancelling...");
         }
     }
+
+    private static boolean acquireUninterruptibly(Semaphore completeSemaphore, 
Span s) {
+        s.reset();
+        return InvokeUtil.getUninterruptibly(() -> {
+            while (!s.elapsed()) {
+                if 
(completeSemaphore.tryAcquire(s.remaining(TimeUnit.NANOSECONDS), 
TimeUnit.NANOSECONDS)) {
+                    return true;
+                }
+            }
+            return false;
+        });
+    }
 }

--
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/19466
To unsubscribe, or for help writing mail filters, visit 
https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: ionic
Gerrit-Change-Id: Ie585127fe30904f5126bae8867b94ea12cd45762
Gerrit-Change-Number: 19466
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <[email protected]>
Gerrit-MessageType: newchange

Reply via email to