Repository: asterixdb
Updated Branches:
  refs/heads/master a26100160 -> 64c021e46


ASTERIXDB-1473: Sporadic test failure in big_ob...

...ject_load_20M

On unsuccessful confiscate cycle, ensure FIFO queue is flushed

Change-Id: I4f932a148dc06bf6ee5142a876f28fa518f34560
Reviewed-on: https://asterix-gerrit.ics.uci.edu/952
Reviewed-by: Jenkins <jenk...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenk...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamou...@gmail.com>


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

Branch: refs/heads/master
Commit: 64c021e46072af275f2df6a91388af3c469e4ba7
Parents: a261001
Author: Michael Blow <mb...@apache.org>
Authored: Tue Jun 21 21:24:24 2016 -0400
Committer: Michael Blow <michael.b...@couchbase.com>
Committed: Tue Jun 21 20:37:34 2016 -0700

----------------------------------------------------------------------
 .../buffercache/AsyncFIFOPageQueueManager.java  | 24 +++++++++++++-------
 .../storage/common/buffercache/BufferCache.java |  2 ++
 .../ClockPageReplacementStrategy.java           |  4 ++--
 3 files changed, 20 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/64c021e4/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 114dcb8..b693c91 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
@@ -89,30 +89,38 @@ public class AsyncFIFOPageQueueManager implements Runnable {
 
     public void destroyQueue(){
         poisoned.set(true);
-        //Dummy cached page to act as poison pill
-        CachedPage poisonPill = new CachedPage();
-        poisonPill.setQueueInfo(new QueueInfo(true,true));
-        if(writerThread == null){
-            synchronized (this){
-                if(writerThread == null) {
+        if (writerThread == null) {
+            synchronized (this) {
+                if (writerThread == null) {
                     return;
                 }
             }
         }
 
+        //Dummy cached page to act as poison pill
+        CachedPage poisonPill = new CachedPage();
+        poisonPill.setQueueInfo(new QueueInfo(true,true));
+
         try{
-            synchronized(poisonPill){
+            synchronized (poisonPill) {
                 queue.put(poisonPill);
                 while(queue.contains(poisonPill)){
                     poisonPill.wait();
                 }
             }
         } catch (InterruptedException e){
-            e.printStackTrace();
+            Thread.currentThread().interrupt();
         }
     }
 
     public void finishQueue() {
+        if (writerThread == null) {
+            synchronized (this) {
+                if (writerThread == null) {
+                    return;
+                }
+            }
+        }
         if(DEBUG) System.out.println("[FIFO] Finishing Queue");
         try {
             //Dummy cached page to act as low water mark

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/64c021e4/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 ceb8ca2..080c76f 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
@@ -473,6 +473,7 @@ public class BufferCache implements IBufferCacheInternal, 
ILifeCycleComponent {
                     Thread.currentThread().interrupt();
                 }
             }
+            finishQueue();
         }
         throw new HyracksDataException("Unable to find free page in buffer 
cache after " + MAX_PIN_ATTEMPT_CYCLES +
                 " cycles (buffer cache undersized?)");
@@ -1314,6 +1315,7 @@ public class BufferCache implements IBufferCacheInternal, 
ILifeCycleComponent {
                     Thread.currentThread().interrupt();
                 }
             }
+            finishQueue();
         }
         throw new HyracksDataException("Unable to find free page in buffer 
cache after " + MAX_PIN_ATTEMPT_CYCLES +
                 " cycles (buffer cache undersized?)");

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/64c021e4/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
----------------------------------------------------------------------
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
index e397d66..c12f288 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/ClockPageReplacementStrategy.java
@@ -130,8 +130,8 @@ public class ClockPageReplacementStrategy implements 
IPageReplacementStrategy {
             }
             if (looped && clockPtr >= startClockPtr) {
                 cycleCount++;
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("completed " + cycleCount + "/" + 
MAX_UNSUCCESSFUL_CYCLE_COUNT
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine("completed " + cycleCount + "/" + 
MAX_UNSUCCESSFUL_CYCLE_COUNT
                             + " clock cycle(s) without finding victim");
                 }
                 if (cycleCount >= MAX_UNSUCCESSFUL_CYCLE_COUNT) {

Reply via email to