[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 <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <imaxon@apache.org>
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 void finishQueue() {
+ public void finishQueue() throws HyracksDataException {
if (writerThread == null) {
synchronized (this) {
if (writerThread == null) {
@@ -121,23 +121,20 @@
}
}
}
- 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
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 @@
}
@Override
- public void finishQueue() {
+ public void finishQueue() throws HyracksDataException {
fifoWriter.finishQueue();
}
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 @@
}
@Override
- public void finishQueue() {
+ public void finishQueue() throws HyracksDataException {
bufferCache.finishQueue();
}
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 @@
/**
* 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
/**