You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2017/11/07 20:25:44 UTC

asterixdb git commit: [ASTERIXDB-1974][STO] Throw Exception on Interrupt

Repository: asterixdb
Updated Branches:
  refs/heads/master 2031eda50 -> 7648e489f


[ASTERIXDB-1974][STO] Throw Exception on Interrupt

- user model changes: no
- storage format changes: no
- interface changes: yes
  - Add HyracksDataException to IBufferCache
  finishQueue.

Details:
- Re-interrupt thread and throw exception when
  an interrupted thread is waiting for its pages
  to be flushed in AsyncFIFOPageQueueManager.

Change-Id: I85197c1f334a5eb2d94d2a305817d280aa619209
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2133
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <im...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/7648e489
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/7648e489
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/7648e489

Branch: refs/heads/master
Commit: 7648e489fe6db03162387d186ebd5ed358c41319
Parents: 2031eda
Author: Murtadha Hubail <mh...@apache.org>
Authored: Tue Nov 7 16:48:34 2017 +0300
Committer: Murtadha Hubail <mh...@apache.org>
Committed: Tue Nov 7 12:25:08 2017 -0800

----------------------------------------------------------------------
 .../buffercache/AsyncFIFOPageQueueManager.java       | 15 ++++++---------
 .../storage/common/buffercache/BufferCache.java      |  2 +-
 .../storage/common/buffercache/DebugBufferCache.java |  2 +-
 .../storage/common/buffercache/IBufferCache.java     |  2 +-
 4 files changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7648e489/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
index d4c1a58..b4f364c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/AsyncFIFOPageQueueManager.java
@@ -113,7 +113,7 @@ public class AsyncFIFOPageQueueManager implements Runnable {
         }
     }
 
-    public void finishQueue() {
+    public void finishQueue() throws HyracksDataException {
         if (writerThread == null) {
             synchronized (this) {
                 if (writerThread == null) {
@@ -121,23 +121,20 @@ public class AsyncFIFOPageQueueManager implements Runnable {
                 }
             }
         }
-        if(DEBUG) System.out.println("[FIFO] Finishing Queue");
         try {
             //Dummy cached page to act as low water mark
             CachedPage lowWater = new CachedPage();
-            lowWater.setQueueInfo(new QueueInfo(true,false));
-            synchronized(lowWater){
+            lowWater.setQueueInfo(new QueueInfo(true, false));
+            synchronized (lowWater) {
                 queue.put(lowWater);
-                while(queue.contains(lowWater)){
-                        lowWater.wait();
+                while (queue.contains(lowWater)) {
+                    lowWater.wait();
                 }
             }
         } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
-            // TODO what do we do here?
-            e.printStackTrace();
+            throw HyracksDataException.create(e);
         }
-        if(DEBUG) System.out.println("[FIFO] Queue finished");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7648e489/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index d0f4965..7167565 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -1379,7 +1379,7 @@ public class BufferCache implements IBufferCacheInternal, ILifeCycleComponent {
     }
 
     @Override
-    public void finishQueue() {
+    public void finishQueue() throws HyracksDataException {
         fifoWriter.finishQueue();
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7648e489/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index f3de1c1..1cbe404 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -199,7 +199,7 @@ public class DebugBufferCache implements IBufferCache {
     }
 
     @Override
-    public void finishQueue() {
+    public void finishQueue() throws HyracksDataException {
         bufferCache.finishQueue();
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7648e489/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index 28801ea..8dccc4a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -235,7 +235,7 @@ public interface IBufferCache {
     /**
      * Flush the queued pages written through buffer cache FIFO queues
      */
-    void finishQueue();
+    void finishQueue() throws HyracksDataException;
 
     // TODO: remove the replication out of the buffer cache interface
     /**