You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by sm...@apache.org on 2015/04/11 10:51:26 UTC

[2/3] drill git commit: DRILL-2741: Continue waiting for completion of sent batches when interrupted

DRILL-2741: Continue waiting for completion of sent batches when interrupted


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/6edf6eb1
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/6edf6eb1
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/6edf6eb1

Branch: refs/heads/master
Commit: 6edf6eb1cf21dc92127cc7cb8d89f77c7edc21fe
Parents: 712b57b
Author: Steven Phillips <sp...@maprtech.com>
Authored: Thu Apr 9 15:31:07 2015 -0700
Committer: Steven Phillips <sp...@maprtech.com>
Committed: Fri Apr 10 23:49:41 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/ops/SendingAccountor.java | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/6edf6eb1/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
index 3d0c90e..0cb5fbf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/SendingAccountor.java
@@ -41,14 +41,14 @@ class SendingAccountor {
   }
 
   public synchronized void waitForSendComplete() {
-    try {
-      int waitForBatches;
-      while((waitForBatches = batchesSent.getAndSet(0)) != 0) {
-        wait.acquire(waitForBatches);
+      int waitForBatches = batchesSent.get();
+      while(waitForBatches != 0) {
+        try {
+          wait.acquire(waitForBatches);
+          waitForBatches = batchesSent.addAndGet(-1 * waitForBatches);
+        } catch (InterruptedException e) {
+          logger.warn("Interrupted while waiting for send complete. Continuing to wait.", e);
+        }
       }
-    } catch (InterruptedException e) {
-      logger.warn("Failure while waiting for send complete.", e);
-      // TODO InterruptedException
-    }
   }
 }