Fixed a bug in the lsm-btree range search cursor where "proceed" is called unnecessarily twice for the first tuple that satisfy the search predicate. Modified the existing search call back test case to cover the fix of locking anti-matter tuples.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_lsm_tree@2948 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 13f7465..7f08ba4 100644
--- a/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -57,7 +57,6 @@
         for (int i = 0; i < rangeCursors.length; i++) {
             pushIntoPriorityQueue(new PriorityQueueElement(i));
         }
-        checkPriorityQueue();
     }
 
     public IIndexCursor getCursor(int cursorIndex) {
diff --git a/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index 8632150..47d00c0 100644
--- a/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -15,8 +15,6 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.rtree.impls;
 
-import java.util.PriorityQueue;
-
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -49,15 +47,6 @@
         super(opCtx);
     }
 
-    public void initPriorityQueue() throws HyracksDataException, IndexException {
-        int pqInitSize = (rangeCursors.length > 0) ? rangeCursors.length : 1;
-        outputPriorityQueue = new PriorityQueue<PriorityQueueElement>(pqInitSize, pqCmp);
-        for (int i = 0; i < rangeCursors.length; i++) {
-            pushIntoPriorityQueue(new PriorityQueueElement(i));
-        }
-        checkPriorityQueue();
-    }
-
     @Override
     public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
diff --git a/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 9341f77..6ce813c 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -67,9 +67,12 @@
         private final RangePredicate predicate;
         private final ArrayTupleBuilder builder;
         private final ArrayTupleReference tuple;
+        private final ArrayTupleBuilder expectedTupleToBeLockedBuilder;
+        private final ArrayTupleReference expectedTupleToBeLocked;
 
         private boolean blockOnHigh;
         private int expectedAfterBlock;
+        private int expectedTupleToBeLockedValue;
 
         public SearchTask() {
             this.cb = new SynchronizingSearchOperationCallback();
@@ -78,9 +81,12 @@
             this.predicate = new RangePredicate();
             this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
             this.tuple = new ArrayTupleReference();
+            this.expectedTupleToBeLockedBuilder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.expectedTupleToBeLocked = new ArrayTupleReference();
 
             this.blockOnHigh = false;
             this.expectedAfterBlock = -1;
+            this.expectedTupleToBeLockedValue = -1;
         }
 
         @Override
@@ -96,6 +102,8 @@
                 predicate.setLowKey(tuple, true);
                 predicate.setHighKey(null, true);
                 accessor.search(cursor, predicate);
+                expectedTupleToBeLockedValue = 50;
+                TupleUtils.createIntegerTuple(builder, expectedTupleToBeLocked, expectedTupleToBeLockedValue);
                 consumeIntTupleRange(50, 75, true, 76);
 
                 // consume tuples [77, 150], blocking on 151
@@ -128,10 +136,6 @@
                     Assert.fail("Failed to consume entire tuple range since cursor is exhausted.");
                 }
                 cursor.next();
-
-                if (this.blockOnHigh) {
-                    TupleUtils.createIntegerTuple(builder, tuple, expectedAfterBlock);
-                }
                 Assert.assertEquals(0, cmp.compare(tuple, cursor.getTuple()));
             }
         }
@@ -140,11 +144,13 @@
 
             @Override
             public boolean proceed(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.expectedTupleToBeLocked, tuple));
                 return false;
             }
 
             @Override
             public void reconcile(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.expectedTupleToBeLocked, tuple));
                 if (blockOnHigh) {
                     try {
                         TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, expectedAfterBlock);
@@ -155,6 +161,14 @@
                     condition.awaitUninterruptibly();
                     blockOnHigh = false;
                 }
+                try {
+                    expectedTupleToBeLockedValue++;
+                    TupleUtils.createIntegerTuple(expectedTupleToBeLockedBuilder, expectedTupleToBeLocked,
+                            expectedTupleToBeLockedValue);
+                } catch (HyracksDataException e) {
+                    e.printStackTrace();
+                }
+
             }
 
             @Override