You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by am...@apache.org on 2018/02/13 00:09:50 UTC

[1/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Repository: asterixdb
Updated Branches:
  refs/heads/master 2f392e855 -> 4ff6a36d1


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeSearchCursorTest.java
new file mode 100644
index 0000000..19b3880
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeSearchCursorTest.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.cursor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.am.lsm.btree.LSMBTreeExamplesTest;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class LSMBTreeSearchCursorTest extends IIndexCursorTest {
+
+    private static final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private static LSMBTree lsmBtree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        lsmBtree = LSMBTreeExamplesTest.createTreeIndex(harness, LSMBTreePointSearchCursorTest.TYPE_TRAITS,
+                LSMBTreePointSearchCursorTest.CMP_FACTORIES, LSMBTreePointSearchCursorTest.BLOOM_FILTER_KEY_FIELDS,
+                null, null, null, null);
+        lsmBtree.create();
+        lsmBtree.activate();
+        LSMBTreePointSearchCursorTest.insertData(lsmBtree);
+    }
+
+    @AfterClass
+    public static void teardown() throws HyracksDataException {
+        try {
+            lsmBtree.deactivate();
+            lsmBtree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws Exception {
+        // exact and windows of length = 50
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            // Build low key.
+            ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(LSMBTreePointSearchCursorTest.KEY_FIELD_COUNT);
+            ArrayTupleReference lowKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(lowKeyTb, lowKey, -100 + (i * 50));
+            // Build high key.
+            ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(LSMBTreePointSearchCursorTest.KEY_FIELD_COUNT);
+            ArrayTupleReference highKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(highKeyTb, highKey, -100 + (i * 50) + 50);
+            MultiComparator lowKeySearchCmp =
+                    BTreeUtils.getSearchMultiComparator(LSMBTreePointSearchCursorTest.CMP_FACTORIES, lowKey);
+            MultiComparator highKeySearchCmp =
+                    BTreeUtils.getSearchMultiComparator(LSMBTreePointSearchCursorTest.CMP_FACTORIES, highKey);
+            predicates.add(new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp));
+            lowKeyTb = new ArrayTupleBuilder(LSMBTreePointSearchCursorTest.KEY_FIELD_COUNT);
+            lowKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(lowKeyTb, lowKey, -100 + (i * 50) + 25);
+            // Build high key.
+            highKeyTb = new ArrayTupleBuilder(LSMBTreePointSearchCursorTest.KEY_FIELD_COUNT);
+            highKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(highKeyTb, highKey, -100 + (i * 50) + 25);
+            lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(LSMBTreePointSearchCursorTest.CMP_FACTORIES, lowKey);
+            highKeySearchCmp =
+                    BTreeUtils.getSearchMultiComparator(LSMBTreePointSearchCursorTest.CMP_FACTORIES, highKey);
+            predicates.add(new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        return lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeSearchCursor.java
index 45e39aa..4e4ecb7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtreeSearchCursor.java
@@ -35,7 +35,7 @@ public class TestLsmBtreeSearchCursor extends LSMBTreeSearchCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         try {
             List<ITestOpCallback<Semaphore>> callbacks = lsmBtree.getSearchCallbacks();
             synchronized (callbacks) {
@@ -47,6 +47,6 @@ public class TestLsmBtreeSearchCursor extends LSMBTreeSearchCursor {
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
-        super.next();
+        super.doNext();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index 5902e62..3423f70 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -260,42 +260,44 @@ public class LSMInvertedIndexTestUtils {
         IInvertedIndexAccessor invIndexAccessor =
                 (IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
         IIndexCursor invIndexCursor = invIndexAccessor.createRangeSearchCursor();
-        MultiComparator tokenCmp = MultiComparator.create(invIndex.getTokenCmpFactories());
-        IBinaryComparatorFactory[] tupleCmpFactories =
-                new IBinaryComparatorFactory[tokenFieldCount + invListFieldCount];
-        for (int i = 0; i < tokenFieldCount; i++) {
-            tupleCmpFactories[i] = invIndex.getTokenCmpFactories()[i];
-        }
-        for (int i = 0; i < invListFieldCount; i++) {
-            tupleCmpFactories[tokenFieldCount + i] = invIndex.getInvListCmpFactories()[i];
-        }
-        MultiComparator tupleCmp = MultiComparator.create(tupleCmpFactories);
-        RangePredicate nullPred = new RangePredicate(null, null, true, true, tokenCmp, tokenCmp);
-        invIndexAccessor.rangeSearch(invIndexCursor, nullPred);
-
-        // Helpers for generating a serialized inverted-list element from a CheckTuple from the expected index.
-        ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
-        ArrayTupleBuilder expectedBuilder = new ArrayTupleBuilder(fieldSerdes.length);
-        ArrayTupleReference expectedTuple = new ArrayTupleReference();
-
-        Iterator<CheckTuple> expectedIter = testCtx.getCheckTuples().iterator();
-
-        // Compare index elements.
         try {
-            while (invIndexCursor.hasNext() && expectedIter.hasNext()) {
-                invIndexCursor.next();
-                ITupleReference actualTuple = invIndexCursor.getTuple();
-                CheckTuple expected = expectedIter.next();
-                OrderedIndexTestUtils.createTupleFromCheckTuple(expected, expectedBuilder, expectedTuple, fieldSerdes);
-                if (tupleCmp.compare(actualTuple, expectedTuple) != 0) {
-                    fail("Index entries differ for token '" + expected.getField(0) + "'.");
-                }
+            MultiComparator tokenCmp = MultiComparator.create(invIndex.getTokenCmpFactories());
+            IBinaryComparatorFactory[] tupleCmpFactories =
+                    new IBinaryComparatorFactory[tokenFieldCount + invListFieldCount];
+            for (int i = 0; i < tokenFieldCount; i++) {
+                tupleCmpFactories[i] = invIndex.getTokenCmpFactories()[i];
             }
-            if (expectedIter.hasNext()) {
-                fail("Indexes do not match. Actual index is missing entries.");
+            for (int i = 0; i < invListFieldCount; i++) {
+                tupleCmpFactories[tokenFieldCount + i] = invIndex.getInvListCmpFactories()[i];
             }
-            if (invIndexCursor.hasNext()) {
-                fail("Indexes do not match. Actual index contains too many entries.");
+            MultiComparator tupleCmp = MultiComparator.create(tupleCmpFactories);
+            RangePredicate nullPred = new RangePredicate(null, null, true, true, tokenCmp, tokenCmp);
+            // Helpers for generating a serialized inverted-list element from a CheckTuple from the expected index.
+            ISerializerDeserializer[] fieldSerdes = testCtx.getFieldSerdes();
+            ArrayTupleBuilder expectedBuilder = new ArrayTupleBuilder(fieldSerdes.length);
+            ArrayTupleReference expectedTuple = new ArrayTupleReference();
+            Iterator<CheckTuple> expectedIter = testCtx.getCheckTuples().iterator();
+            // Compare index elements.
+            invIndexAccessor.rangeSearch(invIndexCursor, nullPred);
+            try {
+                while (invIndexCursor.hasNext() && expectedIter.hasNext()) {
+                    invIndexCursor.next();
+                    ITupleReference actualTuple = invIndexCursor.getTuple();
+                    CheckTuple expected = expectedIter.next();
+                    OrderedIndexTestUtils.createTupleFromCheckTuple(expected, expectedBuilder, expectedTuple,
+                            fieldSerdes);
+                    if (tupleCmp.compare(actualTuple, expectedTuple) != 0) {
+                        fail("Index entries differ for token '" + expected.getField(0) + "'.");
+                    }
+                }
+                if (expectedIter.hasNext()) {
+                    fail("Indexes do not match. Actual index is missing entries.");
+                }
+                if (invIndexCursor.hasNext()) {
+                    fail("Indexes do not match. Actual index contains too many entries.");
+                }
+            } finally {
+                invIndexCursor.close();
             }
         } finally {
             invIndexCursor.destroy();
@@ -517,61 +519,65 @@ public class LSMInvertedIndexTestUtils {
             searchPred.setQueryFieldIndex(0);
 
             IIndexCursor resultCursor = accessor.createSearchCursor(false);
-            boolean panic = false;
             try {
-                accessor.search(resultCursor, searchPred);
-            } catch (HyracksDataException e) {
-                // ignore panic queries.
-                if (e.getErrorCode() == ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
-                    panic = true;
-                } else {
-                    throw e;
+                boolean panic = false;
+                try {
+                    accessor.search(resultCursor, searchPred);
+                } catch (HyracksDataException e) {
+                    // ignore panic queries.
+                    if (e.getErrorCode() == ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
+                        panic = true;
+                    } else {
+                        throw e;
+                    }
                 }
-            }
-
-            try {
-                if (!panic) {
-                    // Consume cursor and deserialize results so we can sort them. Some search cursors may not deliver the result sorted (e.g., LSM search cursor).
-                    ArrayList<Integer> actualResults = new ArrayList<>();
-                    try {
-                        while (resultCursor.hasNext()) {
-                            resultCursor.next();
-                            ITupleReference resultTuple = resultCursor.getTuple();
-                            int actual = IntegerPointable.getInteger(resultTuple.getFieldData(0),
-                                    resultTuple.getFieldStart(0));
-                            actualResults.add(Integer.valueOf(actual));
+                try {
+                    if (!panic) {
+                        // Consume cursor and deserialize results so we can sort them. Some search cursors may not deliver the result sorted (e.g., LSM search cursor).
+                        ArrayList<Integer> actualResults = new ArrayList<>();
+                        try {
+                            while (resultCursor.hasNext()) {
+                                resultCursor.next();
+                                ITupleReference resultTuple = resultCursor.getTuple();
+                                int actual = IntegerPointable.getInteger(resultTuple.getFieldData(0),
+                                        resultTuple.getFieldStart(0));
+                                actualResults.add(Integer.valueOf(actual));
+                            }
+                        } catch (HyracksDataException e) {
+                            if (e.getErrorCode() == ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
+                                // Ignore panic queries.
+                                continue;
+                            } else {
+                                throw e;
+                            }
                         }
-                    } catch (HyracksDataException e) {
-                        if (e.getErrorCode() == ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
-                            // Ignore panic queries.
-                            continue;
-                        } else {
-                            throw e;
+                        Collections.sort(actualResults);
+
+                        // Get expected results.
+                        List<Integer> expectedResults = new ArrayList<>();
+                        LSMInvertedIndexTestUtils.getExpectedResults(scanCountArray, testCtx.getCheckTuples(),
+                                searchDocument, tokenizer, testCtx.getFieldSerdes()[0], searchModifier, expectedResults,
+                                testCtx.getInvertedIndexType());
+
+                        Iterator<Integer> expectedIter = expectedResults.iterator();
+                        Iterator<Integer> actualIter = actualResults.iterator();
+                        while (expectedIter.hasNext() && actualIter.hasNext()) {
+                            int expected = expectedIter.next();
+                            int actual = actualIter.next();
+                            if (actual != expected) {
+                                fail("Query results do not match. Encountered: " + actual + ". Expected: " + expected
+                                        + "");
+                            }
                         }
-                    }
-                    Collections.sort(actualResults);
-
-                    // Get expected results.
-                    List<Integer> expectedResults = new ArrayList<>();
-                    LSMInvertedIndexTestUtils.getExpectedResults(scanCountArray, testCtx.getCheckTuples(),
-                            searchDocument, tokenizer, testCtx.getFieldSerdes()[0], searchModifier, expectedResults,
-                            testCtx.getInvertedIndexType());
-
-                    Iterator<Integer> expectedIter = expectedResults.iterator();
-                    Iterator<Integer> actualIter = actualResults.iterator();
-                    while (expectedIter.hasNext() && actualIter.hasNext()) {
-                        int expected = expectedIter.next();
-                        int actual = actualIter.next();
-                        if (actual != expected) {
-                            fail("Query results do not match. Encountered: " + actual + ". Expected: " + expected + "");
+                        if (expectedIter.hasNext()) {
+                            fail("Query results do not match. Actual results missing.");
+                        }
+                        if (actualIter.hasNext()) {
+                            fail("Query results do not match. Actual contains too many results.");
                         }
                     }
-                    if (expectedIter.hasNext()) {
-                        fail("Query results do not match. Actual results missing.");
-                    }
-                    if (actualIter.hasNext()) {
-                        fail("Query results do not match. Actual contains too many results.");
-                    }
+                } finally {
+                    resultCursor.close();
                 }
             } finally {
                 resultCursor.destroy();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index 947debc..db41659 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -68,6 +68,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-common</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorLifecycleTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorLifecycleTest.java
new file mode 100644
index 0000000..8ae7dac
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorLifecycleTest.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.rtree;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.am.rtree.impls.RTree;
+import org.apache.hyracks.storage.am.rtree.utils.RTreeTestHarness;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class RTreeSearchCursorLifecycleTest extends IIndexCursorTest {
+
+    private static final RTreeTestHarness harness = new RTreeTestHarness();
+    private static RTree rtree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        rtree = new RTree(bufferCache,
+                new LinkedMetaDataPageManager(bufferCache, RTreeSearchCursorTest.META_FRAME_FACTORY),
+                RTreeSearchCursorTest.INTERIOR_FRAME_FACTORY, RTreeSearchCursorTest.LEAF_FRAME_FACTORY,
+                RTreeSearchCursorTest.CMP_FACTORIES, RTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference(),
+                false);
+        rtree.create();
+        rtree.activate();
+        RTreeSearchCursorTest.insert(rtree);
+    }
+
+    @AfterClass
+    public static void teardown() throws HyracksDataException {
+        try {
+            rtree.deactivate();
+            rtree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws Exception {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -100, -100, 100, 100));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -200, -200, 200, 200));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -300, -300, 300, 300));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -400, -400, 400, 400));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -500, -500, 500, 500));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -600, -600, 600, 600));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -700, -700, 700, 700));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -800, -800, 800, 800));
+        predicates.add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -900, -900, 900, 900));
+        predicates
+                .add(RTreeSearchCursorTest.createSearchPredicate(new ArrayTupleReference(), -1000, -1000, 1000, 1000));
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        return rtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index 15f69bc..56c4b53 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -61,8 +61,31 @@ import org.junit.Test;
 
 public class RTreeSearchCursorTest extends AbstractRTreeTest {
 
+    public static final int FIELD_COUNT = 5;
+    public static final ITypeTraits[] TYPE_TRAITS = { IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS,
+            IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS };
+    // Declare field serdes.
+    @SuppressWarnings("rawtypes")
+    public static final ISerializerDeserializer[] FIELD_SERDES = { IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    public static final int KEY_FIELD_COUNT = 4;
+    public static final IBinaryComparatorFactory[] CMP_FACTORIES =
+            { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                    PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                    PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                    PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
+    public static final IPrimitiveValueProviderFactory[] VALUE_PROVIDER_FACTORY =
+            RTreeUtils.createPrimitiveValueProviderFactories(CMP_FACTORIES.length, IntegerPointable.FACTORY);
+    public static final RTreeTypeAwareTupleWriterFactory TUPLE_WRITER_FACTORY =
+            new RTreeTypeAwareTupleWriterFactory(TYPE_TRAITS);
+    public static final ITreeIndexMetadataFrameFactory META_FRAME_FACTORY = new LIFOMetaDataFrameFactory();
+    public static final ITreeIndexFrameFactory INTERIOR_FRAME_FACTORY = new RTreeNSMInteriorFrameFactory(
+            TUPLE_WRITER_FACTORY, VALUE_PROVIDER_FACTORY, RTreePolicyType.RTREE, false);
+    public static final ITreeIndexFrameFactory LEAF_FRAME_FACTORY =
+            new RTreeNSMLeafFrameFactory(TUPLE_WRITER_FACTORY, VALUE_PROVIDER_FACTORY, RTreePolicyType.RTREE, false);
+    private static final Random RND = new Random(50);
     private final RTreeTestUtils rTreeTestUtils;
-    private Random rnd = new Random(50);
 
     public RTreeSearchCursorTest() {
         this.rTreeTestUtils = new RTreeTestUtils();
@@ -74,69 +97,73 @@ public class RTreeSearchCursorTest extends AbstractRTreeTest {
         super.setUp();
     }
 
-    @SuppressWarnings({ "unchecked", "rawtypes" })
+    @SuppressWarnings({ "rawtypes" })
     @Test
     public void rangeSearchTest() throws Exception {
         if (LOGGER.isInfoEnabled()) {
             LOGGER.info("TESTING RANGE SEARCH CURSOR FOR RTREE");
         }
-
         IBufferCache bufferCache = harness.getBufferCache();
-
-        // Declare fields.
-        int fieldCount = 5;
-        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
-        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
-        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
-        typeTraits[2] = IntegerPointable.TYPE_TRAITS;
-        typeTraits[3] = IntegerPointable.TYPE_TRAITS;
-        typeTraits[4] = IntegerPointable.TYPE_TRAITS;
-        // Declare field serdes.
-        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
-
-        // Declare keys.
-        int keyFieldCount = 4;
-        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-        cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-        cmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-        cmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
         // create value providers
-        IPrimitiveValueProviderFactory[] valueProviderFactories =
-                RTreeUtils.createPrimitiveValueProviderFactories(cmpFactories.length, IntegerPointable.FACTORY);
-
-        RTreeTypeAwareTupleWriterFactory tupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(typeTraits);
-        ITreeIndexMetadataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
-        ITreeIndexFrameFactory interiorFrameFactory = new RTreeNSMInteriorFrameFactory(tupleWriterFactory,
-                valueProviderFactories, RTreePolicyType.RTREE, false);
-        ITreeIndexFrameFactory leafFrameFactory =
-                new RTreeNSMLeafFrameFactory(tupleWriterFactory, valueProviderFactories, RTreePolicyType.RTREE, false);
-
-        IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) interiorFrameFactory.createFrame();
-        IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) leafFrameFactory.createFrame();
-        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, metaFrameFactory);
-
-        RTree rtree = new RTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
-                fieldCount, harness.getFileReference(), false);
+        IRTreeInteriorFrame interiorFrame = (IRTreeInteriorFrame) INTERIOR_FRAME_FACTORY.createFrame();
+        IRTreeLeafFrame leafFrame = (IRTreeLeafFrame) LEAF_FRAME_FACTORY.createFrame();
+        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, META_FRAME_FACTORY);
+        RTree rtree = new RTree(bufferCache, freePageManager, INTERIOR_FRAME_FACTORY, LEAF_FRAME_FACTORY, CMP_FACTORIES,
+                FIELD_COUNT, harness.getFileReference(), false);
         rtree.create();
         rtree.activate();
+        ArrayList<RTreeCheckTuple> checkTuples = insert(rtree);
+        ITreeIndexAccessor indexAccessor = rtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        try {
+            // Build key.
+            ArrayTupleReference key = new ArrayTupleReference();
+            SearchPredicate searchPredicate = createSearchPredicate(key, -1000, -1000, 1000, 1000);
+            ITreeIndexCursor searchCursor = new RTreeSearchCursor(interiorFrame, leafFrame);
+            try {
+                RTreeCheckTuple keyCheck =
+                        (RTreeCheckTuple) rTreeTestUtils.createCheckTupleFromTuple(key, FIELD_SERDES, KEY_FIELD_COUNT);
+                HashMultiSet<RTreeCheckTuple> expectedResult =
+                        rTreeTestUtils.getRangeSearchExpectedResults(checkTuples, keyCheck);
+                rTreeTestUtils.getRangeSearchExpectedResults(checkTuples, keyCheck);
+                indexAccessor.search(searchCursor, searchPredicate);
+                try {
+                    rTreeTestUtils.checkExpectedResults(searchCursor, expectedResult, FIELD_SERDES, KEY_FIELD_COUNT,
+                            null);
+                } finally {
+                    searchCursor.close();
+                }
+            } finally {
+                searchCursor.destroy();
+            }
+        } finally {
+            indexAccessor.destroy();
+        }
+        rtree.deactivate();
+        rtree.destroy();
+    }
+
+    public static SearchPredicate createSearchPredicate(ArrayTupleReference key, int first, int second, int third,
+            int fourth) throws HyracksDataException {
+        ArrayTupleBuilder keyTb = new ArrayTupleBuilder(KEY_FIELD_COUNT);
+        TupleUtils.createIntegerTuple(keyTb, key, first, second, third, fourth);
+        MultiComparator cmp = MultiComparator.create(CMP_FACTORIES);
+        return new SearchPredicate(key, cmp);
+    }
 
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    public static ArrayList<RTreeCheckTuple> insert(RTree rtree) throws HyracksDataException {
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(FIELD_COUNT);
         ArrayTupleReference tuple = new ArrayTupleReference();
         ITreeIndexAccessor indexAccessor = rtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
         int numInserts = 10000;
         ArrayList<RTreeCheckTuple> checkTuples = new ArrayList<>();
         for (int i = 0; i < numInserts; i++) {
-            int p1x = rnd.nextInt();
-            int p1y = rnd.nextInt();
-            int p2x = rnd.nextInt();
-            int p2y = rnd.nextInt();
+            int p1x = RND.nextInt();
+            int p1y = RND.nextInt();
+            int p2x = RND.nextInt();
+            int p2y = RND.nextInt();
 
-            int pk = rnd.nextInt();;
+            int pk = RND.nextInt();;
 
             TupleUtils.createIntegerTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                     Math.max(p1y, p2y), pk);
@@ -147,7 +174,7 @@ public class RTreeSearchCursorTest extends AbstractRTreeTest {
                     throw e;
                 }
             }
-            RTreeCheckTuple checkTuple = new RTreeCheckTuple(fieldCount, keyFieldCount);
+            RTreeCheckTuple checkTuple = new RTreeCheckTuple(FIELD_COUNT, KEY_FIELD_COUNT);
             checkTuple.appendField(Math.min(p1x, p2x));
             checkTuple.appendField(Math.min(p1y, p2y));
             checkTuple.appendField(Math.max(p1x, p2x));
@@ -156,28 +183,6 @@ public class RTreeSearchCursorTest extends AbstractRTreeTest {
 
             checkTuples.add(checkTuple);
         }
-
-        // Build key.
-        ArrayTupleBuilder keyTb = new ArrayTupleBuilder(keyFieldCount);
-        ArrayTupleReference key = new ArrayTupleReference();
-        TupleUtils.createIntegerTuple(keyTb, key, -1000, -1000, 1000, 1000);
-
-        MultiComparator cmp = MultiComparator.create(cmpFactories);
-        ITreeIndexCursor searchCursor = new RTreeSearchCursor(interiorFrame, leafFrame);
-        SearchPredicate searchPredicate = new SearchPredicate(key, cmp);
-
-        RTreeCheckTuple keyCheck =
-                (RTreeCheckTuple) rTreeTestUtils.createCheckTupleFromTuple(key, fieldSerdes, keyFieldCount);
-        HashMultiSet<RTreeCheckTuple> expectedResult =
-                rTreeTestUtils.getRangeSearchExpectedResults(checkTuples, keyCheck);
-
-        rTreeTestUtils.getRangeSearchExpectedResults(checkTuples, keyCheck);
-        indexAccessor.search(searchCursor, searchPredicate);
-
-        rTreeTestUtils.checkExpectedResults(searchCursor, expectedResult, fieldSerdes, keyFieldCount, null);
-
-        rtree.deactivate();
-        rtree.destroy();
+        return checkTuples;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
index 9d2d59e..36e0209 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java
@@ -52,37 +52,49 @@ public class RTreeTestWorker extends AbstractIndexTestWorker {
     @Override
     public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException {
         RTree.RTreeAccessor accessor = (RTree.RTreeAccessor) indexAccessor;
-        IIndexCursor searchCursor = accessor.createSearchCursor(false);
-        ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor();
         MultiComparator cmp = accessor.getOpContext().getCmp();
         SearchPredicate rangePred = new SearchPredicate(tuple, cmp);
+        IIndexCursor searchCursor = accessor.createSearchCursor(false);
+        try {
+            ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor();
+            try {
+                switch (op) {
+                    case INSERT:
+                        rearrangeTuple(tuple, cmp);
+                        accessor.insert(rearrangedTuple);
+                        break;
 
-        switch (op) {
-            case INSERT:
-                rearrangeTuple(tuple, cmp);
-                accessor.insert(rearrangedTuple);
-                break;
-
-            case DELETE:
-                rearrangeTuple(tuple, cmp);
-                accessor.delete(rearrangedTuple);
-                break;
-
-            case SCAN:
-                searchCursor.close();
-                rangePred.setSearchKey(null);
-                accessor.search(searchCursor, rangePred);
-                consumeCursorTuples(searchCursor);
-                break;
+                    case DELETE:
+                        rearrangeTuple(tuple, cmp);
+                        accessor.delete(rearrangedTuple);
+                        break;
 
-            case DISKORDER_SCAN:
-                diskOrderScanCursor.close();
-                accessor.diskOrderScan(diskOrderScanCursor);
-                consumeCursorTuples(diskOrderScanCursor);
-                break;
+                    case SCAN:
+                        rangePred.setSearchKey(null);
+                        accessor.search(searchCursor, rangePred);
+                        try {
+                            consumeCursorTuples(searchCursor);
+                        } finally {
+                            searchCursor.close();
+                        }
+                        break;
 
-            default:
-                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+                    case DISKORDER_SCAN:
+                        accessor.diskOrderScan(diskOrderScanCursor);
+                        try {
+                            consumeCursorTuples(diskOrderScanCursor);
+                        } finally {
+                            diskOrderScanCursor.close();
+                        }
+                        break;
+                    default:
+                        throw new HyracksDataException("Op " + op.toString() + " not supported.");
+                }
+            } finally {
+                diskOrderScanCursor.destroy();
+            }
+        } finally {
+            searchCursor.destroy();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 339372e..7b5fd6b 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -16,8 +16,8 @@
  ! specific language governing permissions and limitations
  ! under the License.
  !-->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.hyracks</groupId>
   <artifactId>apache-hyracks</artifactId>
@@ -25,14 +25,12 @@
   <packaging>pom</packaging>
   <name>hyracks-ecosystem-full-stack</name>
   <url>https://asterixdb.apache.org/</url>
-
   <parent>
     <groupId>org.apache</groupId>
     <artifactId>apache</artifactId>
     <version>18</version>
     <relativePath />
   </parent>
-
   <licenses>
     <license>
       <name>Apache License, Version 2.0</name>
@@ -41,14 +39,12 @@
       <comments>A business-friendly OSS license</comments>
     </license>
   </licenses>
-
   <scm>
     <connection>scm:git:https://github.com/apache/asterixdb</connection>
     <developerConnection>scm:git:ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb</developerConnection>
     <url>https://github.com/apache/asterixdb</url>
     <tag>HEAD</tag>
   </scm>
-
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <file.encoding>UTF-8</file.encoding>
@@ -196,9 +192,13 @@
         <artifactId>log4j-core</artifactId>
         <version>2.10.0</version>
       </dependency>
+      <dependency>
+        <groupId>org.mockito</groupId>
+        <artifactId>mockito-all</artifactId>
+        <version>2.0.2-beta</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
-
   <build>
     <plugins>
       <plugin>
@@ -241,10 +241,10 @@
           <addDefaultLicenseMatchers>false</addDefaultLicenseMatchers>
           <consoleOutput>true</consoleOutput>
           <licenses>
-            <license implementation="org.apache.rat.analysis.license.ApacheSoftwareLicense20"/>
+            <license implementation="org.apache.rat.analysis.license.ApacheSoftwareLicense20" />
           </licenses>
           <licenseFamilies>
-            <licenseFamily implementation="org.apache.rat.license.Apache20LicenseFamily"/>
+            <licenseFamily implementation="org.apache.rat.license.Apache20LicenseFamily" />
           </licenseFamilies>
           <excludeSubProjects>true</excludeSubProjects>
           <excludes combine.children="append">
@@ -259,11 +259,8 @@
           <failIfNoTests>false</failIfNoTests>
           <forkCount>1</forkCount>
           <reuseForks>false</reuseForks>
-          <argLine>-enableassertions -Xmx2048m
-            -Dfile.encoding=UTF-8
-            -Xdebug
-            -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n
-            ${coverageArgLine}
+          <argLine>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8 -Xdebug
+            -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${coverageArgLine} 
           </argLine>
           <systemPropertyVariables>
             <log4j.configurationFile>${testLog4jConfigFile}</log4j.configurationFile>
@@ -458,16 +455,16 @@
                 </pluginExecution>
                 <pluginExecution>
                   <pluginExecutionFilter>
-                   <groupId>net.revelc.code.formatter</groupId>
-                   <artifactId>formatter-maven-plugin</artifactId>
-                   <versionRange>[2.0.1,)</versionRange>
-                   <goals>
-                       <goal>format</goal>
-                   </goals>
-                   </pluginExecutionFilter>
-                   <action>
-                       <ignore></ignore>
-                   </action>
+                    <groupId>net.revelc.code.formatter</groupId>
+                    <artifactId>formatter-maven-plugin</artifactId>
+                    <versionRange>[2.0.1,)</versionRange>
+                    <goals>
+                      <goal>format</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
                 </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
@@ -541,7 +538,6 @@
       </plugins>
     </pluginManagement>
   </build>
-
   <profiles>
     <profile>
       <id>skip-assembly</id>
@@ -668,11 +664,9 @@
       </properties>
     </profile>
   </profiles>
-
   <modules>
     <module>hyracks</module>
     <module>algebricks</module>
     <module>hyracks-fullstack-license</module>
   </modules>
-
-</project>
+</project>
\ No newline at end of file


[4/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
index 7bf5322..b2660a4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
@@ -81,7 +81,11 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor {
 
     @Override
     public int compareTo(IInvertedListCursor cursor) {
-        return size() - cursor.size();
+        try {
+            return size() - cursor.size();
+        } catch (HyracksDataException hde) {
+            throw new IllegalStateException(hde);
+        }
     }
 
     public void reset(ITupleReference tuple) throws HyracksDataException {
@@ -113,7 +117,7 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor {
     @Override
     public void unpinPages() throws HyracksDataException {
         if (cursorNeedsClose) {
-            btreeCursor.destroy();
+            btreeCursor.close();
             cursorNeedsClose = false;
         }
     }
@@ -135,29 +139,22 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor {
     }
 
     @Override
-    public int size() {
+    public int size() throws HyracksDataException {
         if (numElements < 0) {
             btreePred.setLowKeyComparator(tokenFieldsCmp);
             btreePred.setHighKeyComparator(tokenFieldsCmp);
             btreePred.setLowKey(tokenTuple, true);
             btreePred.setHighKey(tokenTuple, true);
-
             // Perform the count.
+            btreeAccessor.search(countingCursor, btreePred);
             try {
-                btreeAccessor.search(countingCursor, btreePred);
                 while (countingCursor.hasNext()) {
                     countingCursor.next();
                     ITupleReference countTuple = countingCursor.getTuple();
                     numElements = IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0));
                 }
-            } catch (HyracksDataException e) {
-                e.printStackTrace();
             } finally {
-                try {
-                    countingCursor.destroy();
-                } catch (HyracksDataException e) {
-                    e.printStackTrace();
-                }
+                countingCursor.close();
             }
         }
         return numElements;
@@ -197,7 +194,6 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor {
         try {
             containsKey = btreeCursor.hasNext();
         } finally {
-            btreeCursor.destroy();
             btreeCursor.close();
             btreeSearchTuple.removeLastTuple();
         }
@@ -219,7 +215,6 @@ public class InMemoryInvertedListCursor implements IInvertedListCursor {
                 strBuilder.append(o.toString() + " ");
             }
         } finally {
-            btreeCursor.destroy();
             btreeCursor.close();
         }
         btreeAccessor.search(btreeCursor, btreePred);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
index 75bdbc4..eec2993 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
@@ -70,10 +70,7 @@ public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
 
     @Override
     public boolean hasNext() {
-        if (currentElementIx < numElements)
-            return true;
-        else
-            return false;
+        return currentElementIx < numElements;
     }
 
     @Override
@@ -203,8 +200,9 @@ public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
                 DataInput dataIn = new DataInputStream(inStream);
                 Object o = serdes[i].deserialize(dataIn);
                 strBuilder.append(o.toString());
-                if (i + 1 < tuple.getFieldCount())
+                if (i + 1 < tuple.getFieldCount()) {
                     strBuilder.append(",");
+                }
             }
             strBuilder.append(" ");
         }
@@ -217,6 +215,7 @@ public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
         return strBuilder.toString();
     }
 
+    @Override
     @SuppressWarnings("rawtypes")
     public String printCurrentElement(ISerializerDeserializer[] serdes) throws HyracksDataException {
         StringBuilder strBuilder = new StringBuilder();
@@ -226,8 +225,9 @@ public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
             DataInput dataIn = new DataInputStream(inStream);
             Object o = serdes[i].deserialize(dataIn);
             strBuilder.append(o.toString());
-            if (i + 1 < tuple.getFieldCount())
+            if (i + 1 < tuple.getFieldCount()) {
                 strBuilder.append(",");
+            }
         }
         return strBuilder.toString();
     }
@@ -249,17 +249,23 @@ public class FixedSizeElementInvertedListCursor implements IInvertedListCursor {
             }
         }
 
-        if (begin > arr.length - 1)
+        if (begin > arr.length - 1) {
             return -1;
-        if (key < arr[begin])
+        }
+        if (key < arr[begin]) {
             return begin;
-        else
+        } else {
             return -1;
+        }
     }
 
     @Override
     public int compareTo(IInvertedListCursor invListCursor) {
-        return numElements - invListCursor.size();
+        try {
+            return numElements - invListCursor.size();
+        } catch (HyracksDataException hde) {
+            throw new IllegalStateException(hde);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 4942eda..8e8cb13 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -206,7 +206,6 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
                 listCursor.reset(0, 0, 0, 0);
             }
         } finally {
-            ctx.getBtreeCursor().destroy();
             ctx.getBtreeCursor().close();
         }
     }
@@ -420,6 +419,7 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
         private final OnDiskInvertedIndex index;
         private final IInvertedIndexSearcher searcher;
         private final IIndexOperationContext opCtx = new OnDiskInvertedIndexOpContext(btree);
+        private boolean destroyed = false;
 
         public OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index) throws HyracksDataException {
             this.index = index;
@@ -483,6 +483,15 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
         public void upsert(ITupleReference tuple) throws HyracksDataException {
             throw new UnsupportedOperationException("Upsert not supported by inverted index.");
         }
+
+        @Override
+        public void destroy() throws HyracksDataException {
+            if (destroyed) {
+                return;
+            }
+            destroyed = true;
+            opCtx.destroy();
+        }
     }
 
     @Override
@@ -542,24 +551,38 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
         btree.validate();
         // Scan the btree and validate the order of elements in each inverted-list.
         IIndexAccessor btreeAccessor = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        IIndexCursor btreeCursor = btreeAccessor.createSearchCursor(false);
-        MultiComparator btreeCmp = MultiComparator.create(btree.getComparatorFactories());
-        RangePredicate rangePred = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
+        try {
+            MultiComparator btreeCmp = MultiComparator.create(btree.getComparatorFactories());
+            RangePredicate rangePred = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp);
+            IIndexCursor btreeCursor = btreeAccessor.createSearchCursor(false);
+            try {
+                btreeAccessor.search(btreeCursor, rangePred);
+                try {
+                    doValidate(btreeCursor);
+                } finally {
+                    btreeCursor.close();
+                }
+            } finally {
+                btreeCursor.destroy();
+            }
+        } finally {
+            btreeAccessor.destroy();
+        }
+    }
+
+    private void doValidate(IIndexCursor btreeCursor) throws HyracksDataException {
         int[] fieldPermutation = new int[tokenTypeTraits.length];
         for (int i = 0; i < tokenTypeTraits.length; i++) {
             fieldPermutation[i] = i;
         }
         PermutingTupleReference tokenTuple = new PermutingTupleReference(fieldPermutation);
-
+        // Search key for finding an inverted-list in the actual index.
+        ArrayTupleBuilder prevBuilder = new ArrayTupleBuilder(invListTypeTraits.length);
+        ArrayTupleReference prevTuple = new ArrayTupleReference();
         IInvertedIndexAccessor invIndexAccessor = createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        IInvertedListCursor invListCursor = invIndexAccessor.createInvertedListCursor();
-        MultiComparator invListCmp = MultiComparator.create(invListCmpFactories);
-
         try {
-            // Search key for finding an inverted-list in the actual index.
-            ArrayTupleBuilder prevBuilder = new ArrayTupleBuilder(invListTypeTraits.length);
-            ArrayTupleReference prevTuple = new ArrayTupleReference();
-            btreeAccessor.search(btreeCursor, rangePred);
+            IInvertedListCursor invListCursor = invIndexAccessor.createInvertedListCursor();
+            MultiComparator invListCmp = MultiComparator.create(invListCmpFactories);
             while (btreeCursor.hasNext()) {
                 btreeCursor.next();
                 tokenTuple.reset(btreeCursor.getTuple());
@@ -578,9 +601,7 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
                         invListCursor.next();
                         ITupleReference invListElement = invListCursor.getTuple();
                         // Compare with previous element.
-                        if (invListCmp.compare(invListElement, prevTuple) <= 0) {
-                            throw new HyracksDataException("Index validation failed.");
-                        }
+                        validateWithPrevious(invListCmp, invListElement, prevTuple);
                         // Set new prevTuple.
                         TupleUtils.copyTuple(prevBuilder, invListElement, invListElement.getFieldCount());
                         prevTuple.reset(prevBuilder.getFieldEndOffsets(), prevBuilder.getByteArray());
@@ -590,7 +611,14 @@ public class OnDiskInvertedIndex implements IInPlaceInvertedIndex {
                 }
             }
         } finally {
-            btreeCursor.destroy();
+            invIndexAccessor.destroy();
+        }
+    }
+
+    private void validateWithPrevious(MultiComparator invListCmp, ITupleReference invListElement,
+            ArrayTupleReference prevTuple) throws HyracksDataException {
+        if (invListCmp.compare(invListElement, prevTuple) <= 0) {
+            throw new HyracksDataException("Index validation failed.");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
index 623fc74..89d4e9a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.am.lsm.invertedindex.ondisk;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
@@ -36,6 +37,7 @@ public class OnDiskInvertedIndexOpContext implements IIndexOperationContext {
     private MultiComparator searchCmp;
     // For prefix search on partitioned indexes.
     private MultiComparator prefixSearchCmp;
+    private boolean destroyed = false;
 
     public OnDiskInvertedIndexOpContext(BTree btree) {
         // TODO: Ignore opcallbacks for now.
@@ -82,4 +84,16 @@ public class OnDiskInvertedIndexOpContext implements IIndexOperationContext {
         return prefixSearchCmp;
     }
 
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        try {
+            btreeAccessor.destroy();
+        } finally {
+            btreeCursor.destroy();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
index d1e6531..7af35ff 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -28,6 +28,7 @@ import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -36,7 +37,7 @@ import org.apache.hyracks.storage.common.ISearchPredicate;
 /**
  * Scans a range of tokens, returning tuples containing a token and an inverted-list element.
  */
-public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
+public class OnDiskInvertedIndexRangeSearchCursor extends EnforcedIndexCursor {
 
     private final BTree btree;
     private final IIndexAccessor btreeAccessor;
@@ -69,7 +70,7 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         this.btreePred = (RangePredicate) searchPred;
         btreeAccessor.search(btreeCursor, btreePred);
         invListCursor.pinPages();
@@ -77,7 +78,7 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (invListCursor.hasNext()) {
             return true;
         }
@@ -100,7 +101,7 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         invListCursor.next();
         if (concatTuple.hasMaxTuples()) {
             concatTuple.removeLastTuple();
@@ -109,7 +110,7 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         if (unpinNeeded) {
             invListCursor.unpinPages();
             unpinNeeded = false;
@@ -118,16 +119,16 @@ public class OnDiskInvertedIndexRangeSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         if (unpinNeeded) {
             invListCursor.unpinPages();
             unpinNeeded = false;
         }
-        btreeCursor.destroy();
+        btreeCursor.close();
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return concatTuple;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
index 97f1f76..0563ec9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
@@ -27,11 +27,11 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 
-public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
+public class OnDiskInvertedIndexSearchCursor extends EnforcedIndexCursor {
 
     private List<ByteBuffer> resultBuffers;
     private int numResultBuffers;
@@ -55,7 +55,7 @@ public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         currentBufferIndex = 0;
         tupleIndex = 0;
         resultBuffers = invIndexSearcher.getResultBuffers();
@@ -66,7 +66,7 @@ public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public boolean hasNext() {
+    public boolean doHasNext() {
         if (currentBufferIndex < numResultBuffers && tupleIndex < fta.getTupleCount()) {
             return true;
         } else {
@@ -75,7 +75,7 @@ public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void next() {
+    public void doNext() {
         frameTuple.reset(fta.getBuffer().array(), fta.getTupleStartOffset(tupleIndex));
         resultTuple.reset(frameTuple);
         tupleIndex++;
@@ -89,12 +89,12 @@ public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return resultTuple;
     }
 
     @Override
-    public void close() {
+    public void doClose() {
         currentBufferIndex = 0;
         tupleIndex = 0;
         invIndexSearcher.reset();
@@ -103,7 +103,7 @@ public class OnDiskInvertedIndexSearchCursor implements IIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         currentBufferIndex = 0;
         tupleIndex = 0;
         resultBuffers = null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index 01e0684..064a26d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -87,8 +87,8 @@ public class PartitionedOnDiskInvertedIndex extends OnDiskInvertedIndex implemen
         }
         ctx.getBtreePred().setLowKey(lowSearchKey, true);
         ctx.getBtreePred().setHighKey(highSearchKey, true);
-        ctx.getBtreeAccessor().search(ctx.getBtreeCursor(), ctx.getBtreePred());
         boolean tokenExists = false;
+        ctx.getBtreeAccessor().search(ctx.getBtreeCursor(), ctx.getBtreePred());
         try {
             while (ctx.getBtreeCursor().hasNext()) {
                 ctx.getBtreeCursor().next();
@@ -102,7 +102,6 @@ public class PartitionedOnDiskInvertedIndex extends OnDiskInvertedIndex implemen
                 tokenExists = true;
             }
         } finally {
-            ctx.getBtreeCursor().destroy();
             ctx.getBtreeCursor().close();
         }
         return tokenExists;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
index ff5306c..5ae4e05 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -92,16 +92,14 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
         IPartitionedInvertedIndex partInvIndex = (IPartitionedInvertedIndex) invIndex;
         searchResult.reset();
         if (partInvIndex.isEmpty()) {
+            resultCursor.open(null, searchPred);
             return;
         }
-
         tokenizeQuery(searchPred);
         short numQueryTokens = (short) queryTokenAppender.getTupleCount();
-
         IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
         short numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
         short numTokensUpperBound = searchModifier.getNumTokensUpperBound(numQueryTokens);
-
         occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
         if (occurrenceThreshold <= 0) {
             throw HyracksDataException.create(ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION);
@@ -118,15 +116,14 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
                 maxCountPossible--;
                 // No results possible.
                 if (maxCountPossible < occurrenceThreshold) {
+                    resultCursor.open(null, searchPred);
                     return;
                 }
             }
         }
-
         ArrayList<IInvertedListCursor>[] partitionCursors = partitions.getPartitions();
         short start = partitions.getMinValidPartitionIndex();
         short end = partitions.getMaxValidPartitionIndex();
-
         // Typically, we only enter this case for disk-based inverted indexes.
         // TODO: This behavior could potentially lead to a deadlock if we cannot pin
         // all inverted lists in memory, and are forced to wait for a page to get evicted
@@ -165,7 +162,6 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
             invListMerger.reset();
             invListMerger.merge(partitionCursors[i], occurrenceThreshold, numPrefixLists, searchResult);
         }
-
         resultCursor.open(null, searchPred);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
index 856bf6b..1c118de 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
@@ -16,17 +16,15 @@
  ! specific language governing permissions and limitations
  ! under the License.
  !-->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
-
   <parent>
     <groupId>org.apache.hyracks</groupId>
     <artifactId>hyracks</artifactId>
     <version>0.3.4-SNAPSHOT</version>
   </parent>
-
   <licenses>
     <license>
       <name>Apache License, Version 2.0</name>
@@ -35,11 +33,9 @@
       <comments>A business-friendly OSS license</comments>
     </license>
   </licenses>
-
   <properties>
     <root.dir>${basedir}/../..</root.dir>
   </properties>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -91,5 +87,9 @@
       <artifactId>hyracks-storage-am-bloomfilter</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
   </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java
index 09237a3..9be61fc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTreeOpContext.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.rtree.impls;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
@@ -68,4 +69,9 @@ public class ExternalRTreeOpContext extends AbstractLSMIndexOperationContext {
     public LSMRTreeCursorInitialState getInitialState() {
         return initialState;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        // No Op
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 7172b74..d891d9e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -22,6 +22,8 @@ package org.apache.hyracks.storage.am.lsm.rtree.impls;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang3.mutable.MutableBoolean;
+import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -32,7 +34,6 @@ import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -115,124 +116,247 @@ public class LSMRTree extends AbstractLSMRTree {
         // Renaming order is critical because we use assume ordering when we
         // read the file names when we open the tree.
         // The RTree should be renamed before the BTree.
-
         // scan the memory RTree
-        RTreeAccessor memRTreeAccessor =
-                flushingComponent.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        RTreeSearchCursor rtreeScanCursor = memRTreeAccessor.createSearchCursor(false);
-        SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
-        memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
-
-        ILSMDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(),
-                flushOp.getBTreeTarget(), flushOp.getBloomFilterTarget(), true);
-
-        //count the number of tuples in the buddy btree
+        TreeTupleSorter rTreeTupleSorter = null;
+        MutableBoolean isEmpty = new MutableBoolean(true);
+        MutableLong numBTreeTuples = new MutableLong(0L);
+        ILSMDiskComponent component;
+        ILSMDiskComponentBulkLoader componentBulkLoader = null;
+        RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
         BTreeAccessor memBTreeAccessor =
                 flushingComponent.getBuddyIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
-        IIndexCursor btreeCountingCursor = memBTreeAccessor.createCountingSearchCursor();
-        memBTreeAccessor.search(btreeCountingCursor, btreeNullPredicate);
-        long numBTreeTuples = 0L;
+        boolean abort = true;
         try {
-            while (btreeCountingCursor.hasNext()) {
-                btreeCountingCursor.next();
-                ITupleReference countTuple = btreeCountingCursor.getTuple();
-                numBTreeTuples = IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0));
+            try {
+                rTreeTupleSorter = getRTreeTupleSorter(flushingComponent, memBTreeAccessor, btreeNullPredicate,
+                        numBTreeTuples, isEmpty);
+                rTreeTupleSorter.sort();
+                component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
+                        flushOp.getBloomFilterTarget(), true);
+                componentBulkLoader =
+                        component.createBulkLoader(1.0f, false, numBTreeTuples.longValue(), false, false, false);
+                flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
+                // scan the memory BTree and bulk load delete tuples
+                flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
+            } finally {
+                try {
+                    memBTreeAccessor.destroy();
+                } finally {
+                    if (rTreeTupleSorter != null) {
+                        rTreeTupleSorter.destroy();
+                    }
+                }
             }
-        } finally {
-            btreeCountingCursor.destroy();
-        }
-
-        ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, numBTreeTuples, false, false, false);
-
-        ITreeIndexCursor cursor;
-        IBinaryComparatorFactory[] linearizerArray = { linearizer };
-
-        TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(),
-                linearizerArray, rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(),
-                flushingComponent.getIndex().getBufferCache(), comparatorFields);
-
-        // BulkLoad the tuples from the in-memory tree into the new disk
-        // RTree.
-        boolean isEmpty = true;
-        try {
-            while (rtreeScanCursor.hasNext()) {
-                isEmpty = false;
-                rtreeScanCursor.next();
-                rTreeTupleSorter.insertTupleEntry(rtreeScanCursor.getPageId(), rtreeScanCursor.getTupleOffset());
+            if (component.getLSMComponentFilter() != null) {
+                List<ITupleReference> filterTuples = new ArrayList<>();
+                filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
+                filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
+                getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
+                getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
             }
+            // Note. If we change the filter to write to metadata object, we don't need the if block above
+            flushingComponent.getMetadata().copy(component.getMetadata());
+            abort = false;
+            componentBulkLoader.end();
         } finally {
-            rtreeScanCursor.destroy();
+            if (abort && componentBulkLoader != null) {
+                componentBulkLoader.abort();
+            }
         }
-        rTreeTupleSorter.sort();
-
-        cursor = rTreeTupleSorter;
+        return component;
+    }
 
-        if (!isEmpty) {
+    private void flushLoadRTree(MutableBoolean isEmpty, TreeTupleSorter rTreeTupleSorter,
+            ILSMDiskComponentBulkLoader componentBulkLoader) throws HyracksDataException {
+        if (!isEmpty.booleanValue()) {
+            rTreeTupleSorter.open(null, null);
             try {
-                while (cursor.hasNext()) {
-                    cursor.next();
-                    ITupleReference frameTuple = cursor.getTuple();
+                while (rTreeTupleSorter.hasNext()) {
+                    rTreeTupleSorter.next();
+                    ITupleReference frameTuple = rTreeTupleSorter.getTuple();
                     componentBulkLoader.add(frameTuple);
                 }
             } finally {
-                cursor.destroy();
+                rTreeTupleSorter.close();
             }
         }
+    }
 
-        // scan the memory BTree
+    private void flushLoadBtree(BTreeAccessor memBTreeAccessor, ILSMDiskComponentBulkLoader componentBulkLoader,
+            RangePredicate btreeNullPredicate) throws HyracksDataException {
         IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
-        memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
         try {
-            while (btreeScanCursor.hasNext()) {
-                btreeScanCursor.next();
-                ITupleReference frameTuple = btreeScanCursor.getTuple();
-                componentBulkLoader.delete(frameTuple);
+            memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
+            try {
+                while (btreeScanCursor.hasNext()) {
+                    btreeScanCursor.next();
+                    ITupleReference frameTuple = btreeScanCursor.getTuple();
+                    componentBulkLoader.delete(frameTuple);
+                }
+            } finally {
+                btreeScanCursor.close();
             }
         } finally {
             btreeScanCursor.destroy();
         }
+    }
 
-        if (component.getLSMComponentFilter() != null) {
-            List<ITupleReference> filterTuples = new ArrayList<>();
-            filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
-            filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
-            getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+    private TreeTupleSorter getRTreeTupleSorter(LSMRTreeMemoryComponent flushingComponent,
+            BTreeAccessor memBTreeAccessor, RangePredicate btreeNullPredicate, MutableLong numBTreeTuples,
+            MutableBoolean isEmpty) throws HyracksDataException {
+        RTreeAccessor memRTreeAccessor =
+                flushingComponent.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        try {
+            RTreeSearchCursor rtreeScanCursor = memRTreeAccessor.createSearchCursor(false);
+            try {
+                SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
+                memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
+                try {
+                    //count the number of tuples in the buddy btree
+                    countTuples(memBTreeAccessor, btreeNullPredicate, numBTreeTuples);
+                    IBinaryComparatorFactory[] linearizerArray = { linearizer };
+                    boolean failed = true;
+                    TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(),
+                            linearizerArray, rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(),
+                            flushingComponent.getIndex().getBufferCache(), comparatorFields);
+                    try {
+                        // BulkLoad the tuples from the in-memory tree into the new disk
+                        // RTree.
+                        isEmpty.setValue(fill(rtreeScanCursor, rTreeTupleSorter));
+                        failed = false;
+                    } finally {
+                        if (failed) {
+                            rTreeTupleSorter.destroy();
+                        }
+                    }
+                    return rTreeTupleSorter;
+                } finally {
+                    rtreeScanCursor.close();
+                }
+            } finally {
+                rtreeScanCursor.destroy();
+            }
+        } finally {
+            memRTreeAccessor.destroy();
         }
-        // Note. If we change the filter to write to metadata object, we don't need the if block above
-        flushingComponent.getMetadata().copy(component.getMetadata());
+    }
 
-        componentBulkLoader.end();
-        return component;
+    private boolean fill(RTreeSearchCursor rtreeScanCursor, TreeTupleSorter rTreeTupleSorter)
+            throws HyracksDataException {
+        boolean isEmpty = true;
+        while (rtreeScanCursor.hasNext()) {
+            isEmpty = false;
+            rtreeScanCursor.next();
+            rTreeTupleSorter.insertTupleEntry(rtreeScanCursor.getPageId(), rtreeScanCursor.getTupleOffset());
+        }
+        return isEmpty;
+    }
+
+    private void countTuples(BTreeAccessor memBTreeAccessor, RangePredicate btreeNullPredicate,
+            MutableLong numBTreeTuples) throws HyracksDataException {
+        IIndexCursor btreeCountingCursor = memBTreeAccessor.createCountingSearchCursor();
+        try {
+            memBTreeAccessor.search(btreeCountingCursor, btreeNullPredicate);
+            try {
+                while (btreeCountingCursor.hasNext()) {
+                    btreeCountingCursor.next();
+                    ITupleReference countTuple = btreeCountingCursor.getTuple();
+                    numBTreeTuples.setValue(
+                            IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0)));
+                }
+            } finally {
+                btreeCountingCursor.close();
+            }
+        } finally {
+            btreeCountingCursor.destroy();
+        }
     }
 
     @Override
     public ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException {
         LSMRTreeMergeOperation mergeOp = (LSMRTreeMergeOperation) operation;
         IIndexCursor cursor = mergeOp.getCursor();
+        ILSMDiskComponentBulkLoader componentBulkLoader = null;
+        ILSMDiskComponent mergedComponent = null;
+        boolean abort = true;
+        try {
+            mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), mergeOp.getBTreeTarget(),
+                    mergeOp.getBloomFilterTarget(), true);
+            componentBulkLoader = loadMergeBulkLoader(mergeOp, cursor, mergedComponent);
+            if (mergedComponent.getLSMComponentFilter() != null) {
+                List<ITupleReference> filterTuples = new ArrayList<>();
+                for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                    filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple());
+                    filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
+                }
+                getFilterManager().updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples);
+                getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(),
+                        mergedComponent.getMetadataHolder());
+            }
+            abort = false;
+            componentBulkLoader.end();
+        } finally {
+            try {
+                cursor.destroy();
+            } finally {
+                if (abort && componentBulkLoader != null) {
+                    componentBulkLoader.abort();
+                }
+            }
+        }
+        return mergedComponent;
+    }
+
+    private ILSMDiskComponentBulkLoader loadMergeBulkLoader(LSMRTreeMergeOperation mergeOp, IIndexCursor cursor,
+            ILSMDiskComponent mergedComponent) throws HyracksDataException {
+        ILSMDiskComponentBulkLoader componentBulkLoader = null;
+        boolean abort = true;
         ISearchPredicate rtreeSearchPred = new SearchPredicate(null, null);
         ILSMIndexOperationContext opCtx = ((LSMRTreeSortedCursor) cursor).getOpCtx();
         search(opCtx, cursor, rtreeSearchPred);
-
-        ILSMDiskComponent mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(),
-                mergeOp.getBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
-        ILSMDiskComponentBulkLoader componentBulkLoader;
-        // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the r-trees so that
-        // lsmHarness.endSearch() is called once when the r-trees have been merged.
-        if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
-                .get(diskComponents.size() - 1)) {
-            // Keep the deleted tuples since the oldest disk component is not included in the merge operation
-
-            long numElements = 0L;
-            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
-                        .getNumElements();
+        try {
+            try {
+                // In case we must keep the deleted-keys BTrees, then they must be merged
+                // *before* merging the r-trees so that
+                // lsmHarness.endSearch() is called once when the r-trees have been merged.
+                if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
+                        .get(diskComponents.size() - 1)) {
+                    // Keep the deleted tuples since the oldest disk component
+                    // is not included in the merge operation
+                    long numElements = 0L;
+                    for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                        numElements += ((LSMRTreeDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
+                                .getNumElements();
+                    }
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+                    mergeLoadBTree(opCtx, rtreeSearchPred, componentBulkLoader);
+                } else {
+                    //no buddy-btree needed
+                    componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+                }
+                //search old rtree components
+                while (cursor.hasNext()) {
+                    cursor.next();
+                    ITupleReference frameTuple = cursor.getTuple();
+                    componentBulkLoader.add(frameTuple);
+                }
+            } finally {
+                cursor.close();
+            }
+            abort = false;
+        } finally {
+            if (abort && componentBulkLoader != null) {
+                componentBulkLoader.abort();
             }
-            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+        }
+        return componentBulkLoader;
+    }
 
-            LSMRTreeDeletedKeysBTreeMergeCursor btreeCursor = new LSMRTreeDeletedKeysBTreeMergeCursor(opCtx);
+    private void mergeLoadBTree(ILSMIndexOperationContext opCtx, ISearchPredicate rtreeSearchPred,
+            ILSMDiskComponentBulkLoader componentBulkLoader) throws HyracksDataException {
+        LSMRTreeDeletedKeysBTreeMergeCursor btreeCursor = new LSMRTreeDeletedKeysBTreeMergeCursor(opCtx);
+        try {
             search(opCtx, btreeCursor, rtreeSearchPred);
             try {
                 while (btreeCursor.hasNext()) {
@@ -241,38 +365,11 @@ public class LSMRTree extends AbstractLSMRTree {
                     componentBulkLoader.delete(tuple);
                 }
             } finally {
-                btreeCursor.destroy();
-            }
-        } else {
-            //no buddy-btree needed
-            componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
-        }
-
-        //search old rtree components
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-                ITupleReference frameTuple = cursor.getTuple();
-                componentBulkLoader.add(frameTuple);
+                btreeCursor.close();
             }
         } finally {
-            cursor.destroy();
+            btreeCursor.destroy();
         }
-
-        if (mergedComponent.getLSMComponentFilter() != null) {
-            List<ITupleReference> filterTuples = new ArrayList<>();
-            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple());
-                filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
-            }
-            getFilterManager().updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(),
-                    mergedComponent.getMetadataHolder());
-        }
-
-        componentBulkLoader.end();
-
-        return mergedComponent;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
index e4267e2..d41e406 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java
@@ -41,12 +41,12 @@ import org.apache.hyracks.storage.am.rtree.impls.RTree;
 import org.apache.hyracks.storage.am.rtree.impls.RTree.RTreeAccessor;
 import org.apache.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 
-public abstract class LSMRTreeAbstractCursor implements ILSMIndexCursor {
+public abstract class LSMRTreeAbstractCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
 
     protected boolean open;
     protected RTreeSearchCursor[] rtreeCursors;
@@ -76,7 +76,7 @@ public abstract class LSMRTreeAbstractCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
         if (btreeCmp == null) {
             btreeCmp = lsmInitialState.getBTreeCmp();
@@ -153,7 +153,7 @@ public abstract class LSMRTreeAbstractCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         if (!open) {
             return;
         }
@@ -176,7 +176,7 @@ public abstract class LSMRTreeAbstractCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
index df4e16e..892fe83 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeDeletedKeysBTreeMergeCursor.java
@@ -46,7 +46,7 @@ public class LSMRTreeDeletedKeysBTreeMergeCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
         cmp = lsmInitialState.getBTreeCmp();
         operationalComponents = lsmInitialState.getOperationalComponents();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
index 50f1961..c3c74b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeOpContext.java
@@ -22,6 +22,8 @@ package org.apache.hyracks.storage.am.lsm.rtree.impls;
 import java.util.List;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -51,6 +53,7 @@ public final class LSMRTreeOpContext extends AbstractLSMIndexOperationContext {
     private RTreeOpContext currentRTreeOpContext;
     private BTreeOpContext currentBTreeOpContext;
     private LSMRTreeCursorInitialState searchInitialState;
+    private boolean destroyed = false;
 
     public LSMRTreeOpContext(ILSMIndex index, List<ILSMMemoryComponent> mutableComponents,
             ITreeIndexFrameFactory rtreeLeafFrameFactory, ITreeIndexFrameFactory rtreeInteriorFrameFactory,
@@ -128,4 +131,19 @@ public final class LSMRTreeOpContext extends AbstractLSMIndexOperationContext {
     public RTreeOpContext getCurrentRTreeOpContext() {
         return currentRTreeOpContext;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        Throwable failure = DestroyUtils.destroy(null, mutableRTreeAccessors);
+        failure = DestroyUtils.destroy(failure, rtreeOpContexts);
+        failure = DestroyUtils.destroy(failure, mutableBTreeAccessors);
+        failure = DestroyUtils.destroy(failure, btreeOpContexts);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index 0edf22f..d485f64 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -39,13 +39,13 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        super.destroy();
+    public void doDestroy() throws HyracksDataException {
+        super.doDestroy();
         currentCursor = 0;
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         if (!open) {
             return;
         }
@@ -54,8 +54,8 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
         foundNext = false;
         try {
             for (int i = 0; i < numberOfTrees; i++) {
-                rtreeCursors[i].destroy();
-                btreeCursors[i].destroy();
+                rtreeCursors[i].close();
+                btreeCursors[i].close();
             }
             rtreeCursors = null;
             btreeCursors = null;
@@ -88,7 +88,7 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (foundNext) {
             return true;
         }
@@ -111,7 +111,7 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
                             killerTupleFound = true;
                         }
                     } finally {
-                        btreeCursors[i].destroy();
+                        btreeCursors[i].close();
                     }
                 }
                 if (!killerTupleFound) {
@@ -120,7 +120,7 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
                     return true;
                 }
             }
-            rtreeCursors[currentCursor].destroy();
+            rtreeCursors[currentCursor].close();
             currentCursor++;
             searchNextCursor();
         }
@@ -128,13 +128,13 @@ public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        super.open(initialState, searchPred);
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        super.doOpen(initialState, searchPred);
         searchNextCursor();
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
index d66e882..9bbc3e1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
@@ -51,17 +51,21 @@ public class LSMRTreeSortedCursor extends LSMRTreeAbstractCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         depletedRtreeCursors = new boolean[numberOfTrees];
         foundNext = false;
         try {
             for (int i = 0; i < numberOfTrees; i++) {
                 rtreeCursors[i].close();
                 rtreeAccessors[i].search(rtreeCursors[i], rtreeSearchPredicate);
-                if (rtreeCursors[i].hasNext()) {
-                    rtreeCursors[i].next();
-                } else {
-                    depletedRtreeCursors[i] = true;
+                try {
+                    if (rtreeCursors[i].hasNext()) {
+                        rtreeCursors[i].next();
+                    } else {
+                        depletedRtreeCursors[i] = true;
+                    }
+                } finally {
+                    rtreeCursors[i].close();
                 }
             }
         } finally {
@@ -88,7 +92,7 @@ public class LSMRTreeSortedCursor extends LSMRTreeAbstractCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         while (!foundNext) {
             frameTuple = null;
 
@@ -138,7 +142,7 @@ public class LSMRTreeSortedCursor extends LSMRTreeAbstractCursor {
                         break;
                     }
                 } finally {
-                    btreeCursors[i].destroy();
+                    btreeCursors[i].close();
                 }
             }
             if (!killed) {
@@ -150,14 +154,13 @@ public class LSMRTreeSortedCursor extends LSMRTreeAbstractCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        super.open(initialState, searchPred);
-
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        super.doOpen(initialState, searchPred);
         depletedRtreeCursors = new boolean[numberOfTrees];
         foundNext = false;
         for (int i = 0; i < numberOfTrees; i++) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 94d76f6..cade80f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -92,85 +92,133 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
         // read the file names when we open the tree.
         // The RTree should be renamed before the BTree.
         LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent();
-        RTreeAccessor memRTreeAccessor =
-                flushingComponent.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        RTreeSearchCursor rtreeScanCursor = memRTreeAccessor.createSearchCursor(false);
         SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null);
-        memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
-        ILSMDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
-        ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, 0L, false, false, false);
-
-        // Since the LSM-RTree is used as a secondary assumption, the
-        // primary key will be the last comparator in the BTree comparators
-        TreeTupleSorter rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(),
-                linearizerArray, rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(),
-                flushingComponent.getIndex().getBufferCache(), comparatorFields);
-
+        ILSMDiskComponent component = null;
+        ILSMDiskComponentBulkLoader componentBulkLoader = null;
+        TreeTupleSorter rTreeTupleSorter = null;
+        TreeTupleSorter bTreeTupleSorter = null;
         boolean isEmpty = true;
+        boolean abort = true;
         try {
-            while (rtreeScanCursor.hasNext()) {
-                isEmpty = false;
-                rtreeScanCursor.next();
-                rTreeTupleSorter.insertTupleEntry(rtreeScanCursor.getPageId(), rtreeScanCursor.getTupleOffset());
-            }
-        } finally {
-            rtreeScanCursor.destroy();
-        }
-        if (!isEmpty) {
-            rTreeTupleSorter.sort();
-        }
+            RTreeAccessor memRTreeAccessor =
+                    flushingComponent.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
 
-        // scan the memory BTree
-        BTreeAccessor memBTreeAccessor =
-                flushingComponent.getBuddyIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
-        RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
-        memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
-        TreeTupleSorter bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(),
-                linearizerArray, btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
-                flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
-
-        isEmpty = true;
-        try {
-            while (btreeScanCursor.hasNext()) {
-                isEmpty = false;
-                btreeScanCursor.next();
-                bTreeTupleSorter.insertTupleEntry(btreeScanCursor.getPageId(), btreeScanCursor.getTupleOffset());
+            try {
+                RTreeSearchCursor rtreeScanCursor = memRTreeAccessor.createSearchCursor(false);
+                try {
+                    memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
+                    component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
+                    componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
+                    // Since the LSM-RTree is used as a secondary assumption, the
+                    // primary key will be the last comparator in the BTree comparators
+                    rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(), linearizerArray,
+                            rtreeLeafFrameFactory.createFrame(), rtreeLeafFrameFactory.createFrame(),
+                            flushingComponent.getIndex().getBufferCache(), comparatorFields);
+                    try {
+                        isEmpty = scanAndSort(rtreeScanCursor, rTreeTupleSorter);
+                    } finally {
+                        rtreeScanCursor.close();
+                    }
+                } finally {
+                    rtreeScanCursor.destroy();
+                }
+            } finally {
+                memRTreeAccessor.destroy();
             }
-        } finally {
-            btreeScanCursor.destroy();
-        }
-        if (!isEmpty) {
-            bTreeTupleSorter.sort();
-        }
-
-        LSMRTreeWithAntiMatterTuplesFlushCursor cursor = new LSMRTreeWithAntiMatterTuplesFlushCursor(rTreeTupleSorter,
-                bTreeTupleSorter, comparatorFields, linearizerArray);
-        cursor.open(null, null);
-
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-                ITupleReference frameTuple = cursor.getTuple();
-
-                componentBulkLoader.add(frameTuple);
+            if (!isEmpty) {
+                rTreeTupleSorter.sort();
+            }
+            // scan the memory BTree
+            RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null);
+            BTreeAccessor memBTreeAccessor =
+                    flushingComponent.getBuddyIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
+            try {
+                bTreeTupleSorter = new TreeTupleSorter(flushingComponent.getBuddyIndex().getFileId(), linearizerArray,
+                        btreeLeafFrameFactory.createFrame(), btreeLeafFrameFactory.createFrame(),
+                        flushingComponent.getBuddyIndex().getBufferCache(), comparatorFields);
+                BTreeRangeSearchCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false);
+                try {
+                    isEmpty = true;
+                    memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate);
+                    try {
+                        isEmpty = scanAndSort(btreeScanCursor, bTreeTupleSorter);
+                    } finally {
+                        btreeScanCursor.close();
+                    }
+                } finally {
+                    btreeScanCursor.destroy();
+                }
+            } finally {
+                memBTreeAccessor.destroy();
+            }
+            if (!isEmpty) {
+                bTreeTupleSorter.sort();
+            }
+            LSMRTreeWithAntiMatterTuplesFlushCursor cursor = new LSMRTreeWithAntiMatterTuplesFlushCursor(
+                    rTreeTupleSorter, bTreeTupleSorter, comparatorFields, linearizerArray);
+            try {
+                cursor.open(null, null);
+                try {
+                    while (cursor.hasNext()) {
+                        cursor.next();
+                        ITupleReference frameTuple = cursor.getTuple();
+                        componentBulkLoader.add(frameTuple);
+                    }
+                } finally {
+                    cursor.close();
+                }
+            } finally {
+                cursor.destroy();
+            }
+            if (component.getLSMComponentFilter() != null) {
+                List<ITupleReference> filterTuples = new ArrayList<>();
+                filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
+                filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
+                getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
+                getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
             }
+            flushingComponent.getMetadata().copy(component.getMetadata());
+            abort = false;
+            componentBulkLoader.end();
         } finally {
-            cursor.destroy();
+            try {
+                if (rTreeTupleSorter != null) {
+                    rTreeTupleSorter.destroy();
+                }
+            } finally {
+                try {
+                    if (bTreeTupleSorter != null) {
+                        bTreeTupleSorter.destroy();
+                    }
+                } finally {
+                    if (abort && componentBulkLoader != null) {
+                        componentBulkLoader.abort();
+                    }
+                }
+            }
         }
+        return component;
+    }
 
-        if (component.getLSMComponentFilter() != null) {
-            List<ITupleReference> filterTuples = new ArrayList<>();
-            filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
-            filterTuples.add(flushingComponent.getLSMComponentFilter().getMaxTuple());
-            getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+    private boolean scanAndSort(RTreeSearchCursor scanCursor, TreeTupleSorter tupleSorter) throws HyracksDataException {
+        boolean isEmpty = true;
+        while (scanCursor.hasNext()) {
+            isEmpty = false;
+            scanCursor.next();
+            tupleSorter.insertTupleEntry(scanCursor.getPageId(), scanCursor.getTupleOffset());
         }
-        flushingComponent.getMetadata().copy(component.getMetadata());
+        return isEmpty;
+    }
 
-        componentBulkLoader.end();
-        return component;
+    private boolean scanAndSort(BTreeRangeSearchCursor scanCursor, TreeTupleSorter tupleSorter)
+            throws HyracksDataException {
+        boolean isEmpty = true;
+        while (scanCursor.hasNext()) {
+            isEmpty = false;
+            scanCursor.next();
+            tupleSorter.insertTupleEntry(scanCursor.getPageId(), scanCursor.getTupleOffset());
+        }
+        return isEmpty;
     }
 
     @Override
@@ -193,7 +241,7 @@ public class LSMRTreeWithAntiMatterTuples extends AbstractLSMRTree {
                 componentBulkLoader.add(frameTuple);
             }
         } finally {
-            cursor.destroy();
+            cursor.close();
         }
         if (component.getLSMComponentFilter() != null) {
             List<ITupleReference> filterTuples = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
index c2a2bf5..77bf58e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
@@ -23,12 +23,12 @@ import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 
-public class LSMRTreeWithAntiMatterTuplesFlushCursor implements ILSMIndexCursor {
+public class LSMRTreeWithAntiMatterTuplesFlushCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
     private final TreeTupleSorter rTreeTupleSorter;
     private final TreeTupleSorter bTreeTupleSorter;
     private final int[] comparatorFields;
@@ -48,12 +48,23 @@ public class LSMRTreeWithAntiMatterTuplesFlushCursor implements ILSMIndexCursor
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        boolean rtreeOpen = false;
+        boolean btreeOpen = false;
+        try {
+            rTreeTupleSorter.open(initialState, searchPred);
+            rtreeOpen = true;
+            bTreeTupleSorter.open(initialState, searchPred);
+            btreeOpen = true;
+        } finally {
+            if (rtreeOpen && !btreeOpen) {
+                rTreeTupleSorter.close();
+            }
+        }
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (foundNext) {
             return true;
         }
@@ -124,22 +135,31 @@ public class LSMRTreeWithAntiMatterTuplesFlushCursor implements ILSMIndexCursor
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
 
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
+        try {
+            rTreeTupleSorter.destroy();
+        } finally {
+            bTreeTupleSorter.destroy();
+        }
     }
 
     @Override
-    public void close() throws HyracksDataException {
-
+    public void doClose() throws HyracksDataException {
+        try {
+            rTreeTupleSorter.close();
+        } finally {
+            bTreeTupleSorter.close();
+        }
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index b8180a6..4547063 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -70,7 +70,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMRTreeCursorInitialState lsmInitialState = (LSMRTreeCursorInitialState) initialState;
         cmp = lsmInitialState.getHilbertCmp();
         btreeCmp = lsmInitialState.getBTreeCmp();
@@ -136,7 +136,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (numMemoryComponents > 0) {
             if (foundNext) {
                 return true;
@@ -158,13 +158,13 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                         return true;
                     }
                 }
-                mutableRTreeCursors[currentCursor].destroy();
+                mutableRTreeCursors[currentCursor].close();
                 currentCursor++;
                 searchNextCursor();
             }
-            while (super.hasNext()) {
-                super.next();
-                ITupleReference diskRTreeTuple = super.getTuple();
+            while (super.doHasNext()) {
+                super.doNext();
+                ITupleReference diskRTreeTuple = super.doGetTuple();
                 // TODO: at this time, we only add proceed().
                 // reconcile() and complete() can be added later after considering the semantics.
 
@@ -178,13 +178,11 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                 }
             }
         } else {
-            if (super.hasNext()) {
-                super.next();
-                ITupleReference diskRTreeTuple = super.getTuple();
-
+            if (super.doHasNext()) {
+                super.doNext();
+                ITupleReference diskRTreeTuple = super.doGetTuple();
                 // TODO: at this time, we only add proceed() part.
                 // reconcile() and complete() can be added later after considering the semantics.
-
                 // Call proceed() to do necessary operations before returning this tuple.
                 // Since in-memory components don't exist, we can skip searching in-memory B-Trees.
                 searchCallback.proceed(diskRTreeTuple);
@@ -214,17 +212,17 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         if (!open) {
             return;
         }
@@ -236,11 +234,11 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                 btreeCursors[i].close();
             }
         }
-        super.close();
+        super.doClose();
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         if (!open) {
             return;
         }
@@ -252,7 +250,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
         }
         currentCursor = 0;
         open = false;
-        super.destroy();
+        super.doDestroy();
     }
 
     @Override
@@ -272,7 +270,7 @@ public class LSMRTreeWithAntiMatterTuplesSearchCursor extends LSMIndexSearchCurs
                     return false;
                 }
             } finally {
-                btreeCursors[i].destroy();
+                btreeCursors[i].close();
             }
         }
         return true;


[5/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index b4990d6..4610fd2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -180,22 +180,24 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
         predicate.setLowKey(tuple);
         if (needKeyDupCheck) {
             // first check the inmemory component
+            boolean found;
             ctx.getCurrentMutableBTreeAccessor().search(memCursor, predicate);
             try {
-                if (memCursor.hasNext()) {
+                found = memCursor.hasNext();
+                if (found) {
                     memCursor.next();
                     LSMBTreeTupleReference lsmbtreeTuple = (LSMBTreeTupleReference) memCursor.getTuple();
                     if (!lsmbtreeTuple.isAntimatter()) {
                         throw HyracksDataException.create(ErrorCode.DUPLICATE_KEY);
-                    } else {
-                        memCursor.destroy();
-                        ctx.getCurrentMutableBTreeAccessor().upsertIfConditionElseInsert(tuple,
-                                AntimatterAwareTupleAcceptor.INSTANCE);
-                        return true;
                     }
                 }
             } finally {
-                memCursor.destroy();
+                memCursor.close();
+            }
+            if (found) {
+                ctx.getCurrentMutableBTreeAccessor().upsertIfConditionElseInsert(tuple,
+                        AntimatterAwareTupleAcceptor.INSTANCE);
+                return true;
             }
 
             // TODO: Can we just remove the above code that search the mutable
@@ -213,7 +215,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
                     throw HyracksDataException.create(ErrorCode.DUPLICATE_KEY);
                 }
             } finally {
-                searchCursor.destroy();
+                searchCursor.close();
                 // Add the current active mutable component back
                 ctx.getComponentHolder().add(0, firstComponent);
             }
@@ -241,7 +243,8 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
         MultiComparator comp = MultiComparator.create(getComparatorFactories());
         ISearchPredicate pred = new RangePredicate(null, null, true, true, comp, comp);
         ctx.getSearchInitialState().reset(pred, operationalComponents);
-        ((LSMBTreeSearchCursor) cursor).scan(ctx.getSearchInitialState(), pred);
+        ctx.getSearchInitialState().setDiskComponentScan(true);
+        ((LSMBTreeSearchCursor) cursor).open(ctx.getSearchInitialState(), pred);
     }
 
     @Override
@@ -249,46 +252,55 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
         LSMBTreeFlushOperation flushOp = (LSMBTreeFlushOperation) operation;
         LSMBTreeMemoryComponent flushingComponent = (LSMBTreeMemoryComponent) flushOp.getFlushingComponent();
         IIndexAccessor accessor = flushingComponent.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
-
-        RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
-        long numElements = 0L;
-        if (hasBloomFilter) {
-            //count elements in btree for creating Bloomfilter
-            IIndexCursor countingCursor = ((BTreeAccessor) accessor).createCountingSearchCursor();
-            accessor.search(countingCursor, nullPred);
+        ILSMDiskComponent component;
+        ILSMDiskComponentBulkLoader componentBulkLoader;
+        try {
+            RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
+            long numElements = 0L;
+            if (hasBloomFilter) {
+                //count elements in btree for creating Bloomfilter
+                IIndexCursor countingCursor = ((BTreeAccessor) accessor).createCountingSearchCursor();
+                accessor.search(countingCursor, nullPred);
+                try {
+                    while (countingCursor.hasNext()) {
+                        countingCursor.next();
+                        ITupleReference countTuple = countingCursor.getTuple();
+                        numElements =
+                                IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0));
+                    }
+                } finally {
+                    try {
+                        countingCursor.close();
+                    } finally {
+                        countingCursor.destroy();
+                    }
+                }
+            }
+            component = createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(),
+                    true);
+            componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
+            IIndexCursor scanCursor = accessor.createSearchCursor(false);
+            accessor.search(scanCursor, nullPred);
             try {
-                while (countingCursor.hasNext()) {
-                    countingCursor.next();
-                    ITupleReference countTuple = countingCursor.getTuple();
-                    numElements = IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0));
+                while (scanCursor.hasNext()) {
+                    scanCursor.next();
+                    // we can safely throw away updated tuples in secondary BTree components, because they correspond to
+                    // deleted tuples
+                    if (updateAware && ((LSMBTreeTupleReference) scanCursor.getTuple()).isUpdated()) {
+                        continue;
+                    }
+                    componentBulkLoader.add(scanCursor.getTuple());
                 }
             } finally {
-                countingCursor.destroy();
-            }
-        }
-
-        ILSMDiskComponent component =
-                createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(), true);
-
-        ILSMDiskComponentBulkLoader componentBulkLoader =
-                component.createBulkLoader(1.0f, false, numElements, false, false, false);
-
-        IIndexCursor scanCursor = accessor.createSearchCursor(false);
-        accessor.search(scanCursor, nullPred);
-        try {
-            while (scanCursor.hasNext()) {
-                scanCursor.next();
-                // we can safely throw away updated tuples in secondary BTree components, because they correspond to
-                // deleted tuples
-                if (updateAware && ((LSMBTreeTupleReference) scanCursor.getTuple()).isUpdated()) {
-                    continue;
+                try {
+                    scanCursor.close();
+                } finally {
+                    scanCursor.destroy();
                 }
-                componentBulkLoader.add(scanCursor.getTuple());
             }
         } finally {
-            scanCursor.destroy();
+            accessor.destroy();
         }
-
         if (component.getLSMComponentFilter() != null) {
             List<ITupleReference> filterTuples = new ArrayList<>();
             filterTuples.add(flushingComponent.getLSMComponentFilter().getMinTuple());
@@ -313,9 +325,55 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
     public ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException {
         LSMBTreeMergeOperation mergeOp = (LSMBTreeMergeOperation) operation;
         IIndexCursor cursor = mergeOp.getCursor();
-        RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
-        search(mergeOp.getAccessor().getOpContext(), cursor, rangePred);
-        List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
+        ILSMDiskComponent mergedComponent;
+        ILSMDiskComponentBulkLoader componentBulkLoader = null;
+        try {
+            try {
+                RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+                search(mergeOp.getAccessor().getOpContext(), cursor, rangePred);
+                try {
+                    List<ILSMComponent> mergedComponents = mergeOp.getMergingComponents();
+                    long numElements = getNumberOfElements(mergedComponents);
+                    mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
+                            mergeOp.getBloomFilterTarget(), true);
+                    componentBulkLoader =
+                            mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+                    while (cursor.hasNext()) {
+                        cursor.next();
+                        ITupleReference frameTuple = cursor.getTuple();
+                        componentBulkLoader.add(frameTuple);
+                    }
+                } finally {
+                    cursor.close();
+                }
+            } finally {
+                cursor.destroy();
+            }
+            if (mergedComponent.getLSMComponentFilter() != null) {
+                List<ITupleReference> filterTuples = new ArrayList<>();
+                for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                    filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple());
+                    filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
+                }
+                getFilterManager().updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples);
+                getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(),
+                        mergedComponent.getMetadataHolder());
+            }
+        } catch (Throwable e) { // NOSONAR.. As per the contract, we should either abort or end
+            try {
+                if (componentBulkLoader != null) {
+                    componentBulkLoader.abort();
+                }
+            } catch (Throwable th) { // NOSONAR Don't lose the root failure
+                e.addSuppressed(th);
+            }
+            throw e;
+        }
+        componentBulkLoader.end();
+        return mergedComponent;
+    }
+
+    private long getNumberOfElements(List<ILSMComponent> mergedComponents) throws HyracksDataException {
         long numElements = 0L;
         if (hasBloomFilter) {
             //count elements in btree for creating Bloomfilter
@@ -324,32 +382,7 @@ public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
                         .getNumElements();
             }
         }
-        ILSMDiskComponent mergedComponent =
-                createDiskComponent(componentFactory, mergeOp.getTarget(), null, mergeOp.getBloomFilterTarget(), true);
-
-        ILSMDiskComponentBulkLoader componentBulkLoader =
-                mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-                ITupleReference frameTuple = cursor.getTuple();
-                componentBulkLoader.add(frameTuple);
-            }
-        } finally {
-            cursor.destroy();
-        }
-        if (mergedComponent.getLSMComponentFilter() != null) {
-            List<ITupleReference> filterTuples = new ArrayList<>();
-            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple());
-                filterTuples.add(mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple());
-            }
-            getFilterManager().updateFilter(mergedComponent.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(mergedComponent.getLSMComponentFilter(),
-                    mergedComponent.getMetadataHolder());
-        }
-        componentBulkLoader.end();
-        return mergedComponent;
+        return numElements;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
index dd1beee..9e8c848 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeCursorInitialState.java
@@ -36,11 +36,10 @@ public class LSMBTreeCursorInitialState implements ICursorInitialState {
     private MultiComparator cmp;
     private final MultiComparator bloomFilterCmp;
     private final ILSMHarness lsmHarness;
-
     private ISearchPredicate predicate;
     private ISearchOperationCallback searchCallback;
-
     private List<ILSMComponent> operationalComponents;
+    private boolean isDiskComponentScan;
 
     public LSMBTreeCursorInitialState(ITreeIndexFrameFactory leafFrameFactory, MultiComparator cmp,
             MultiComparator bloomFilterCmp, ILSMHarness lsmHarness, ISearchPredicate predicate,
@@ -105,7 +104,16 @@ public class LSMBTreeCursorInitialState implements ICursorInitialState {
 
     // make the cursor initial state re-usable
     public void reset(ISearchPredicate predicate, List<ILSMComponent> operationalComponents) {
+        isDiskComponentScan = false;
         this.predicate = predicate;
         this.operationalComponents = operationalComponents;
     }
+
+    public void setDiskComponentScan(boolean isDiskComponentScan) {
+        this.isDiskComponentScan = isDiskComponentScan;
+    }
+
+    public boolean isDiskComponentScan() {
+        return isDiskComponentScan;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
index a789ddd..dacd41f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeDiskComponentScanCursor.java
@@ -20,6 +20,8 @@
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.data.std.primitive.BooleanPointable;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
@@ -61,7 +63,7 @@ public class LSMBTreeDiskComponentScanCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
         cmp = lsmInitialState.getOriginalKeyComparator();
         operationalComponents = lsmInitialState.getOperationalComponents();
@@ -87,18 +89,18 @@ public class LSMBTreeDiskComponentScanCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (foundNext) {
             return true;
         }
-        while (super.hasNext()) {
-            super.next();
-            LSMBTreeTupleReference diskTuple = (LSMBTreeTupleReference) super.getTuple();
+        while (super.doHasNext()) {
+            super.doNext();
+            LSMBTreeTupleReference diskTuple = (LSMBTreeTupleReference) super.doGetTuple();
             if (diskTuple.isAntimatter()) {
                 if (setAntiMatterTuple(diskTuple, outputElement.getCursorIndex())) {
                     foundNext = true;
@@ -167,23 +169,28 @@ public class LSMBTreeDiskComponentScanCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return outputTuple;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
+        Throwable failure = null;
         if (lsmHarness != null) {
-            try {
-                for (int i = 0; i < rangeCursors.length; i++) {
-                    rangeCursors[i].destroy();
-                }
+            if (rangeCursors != null) {
+                failure = DestroyUtils.destroy(failure, rangeCursors);
                 rangeCursors = null;
-            } finally {
+            }
+            try {
                 lsmHarness.endScanDiskComponents(opCtx);
+            } catch (Throwable th) { // NOSONAR. Don't lose the root cause
+                failure = ExceptionUtils.suppress(failure, th);
             }
         }
         foundNext = false;
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 9cab94e..7795075 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -22,6 +22,8 @@ package org.apache.hyracks.storage.am.lsm.btree.impls;
 import java.util.List;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTreeOpContext;
@@ -63,6 +65,7 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
      */
     private BTree.BTreeAccessor currentMutableBTreeAccessor;
     private BTreeOpContext currentMutableBTreeOpCtx;
+    private boolean destroyed = false;
 
     public LSMBTreeOpContext(ILSMIndex index, List<ILSMMemoryComponent> mutableComponents,
             ITreeIndexFrameFactory insertLeafFrameFactory, ITreeIndexFrameFactory deleteLeafFrameFactory,
@@ -176,4 +179,18 @@ public final class LSMBTreeOpContext extends AbstractLSMIndexOperationContext {
     public MultiComparator getCmp() {
         return cmp;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        Throwable failure = DestroyUtils.destroy(null, mutableBTreeAccessors);
+        failure = DestroyUtils.destroy(failure, mutableBTreeOpCtxs);
+        failure = DestroyUtils.destroy(failure, insertSearchCursor, memCursor);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index 0c39a11..78564fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -37,12 +37,12 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 
-public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
+public class LSMBTreePointSearchCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
 
     private ITreeIndexCursor[] btreeCursors;
     private final ILSMIndexOperationContext opCtx;
@@ -66,7 +66,7 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (nextHasBeenCalled) {
             return false;
         } else if (foundTuple) {
@@ -88,7 +88,7 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
                         if (reconciled) {
                             searchCallback.cancel(predicate.getLowKey());
                         }
-                        btreeCursors[i].destroy();
+                        btreeCursors[i].close();
                         return false;
                     } else {
                         frameTuple = btreeCursors[i].getTuple();
@@ -109,7 +109,7 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
                         btreeCursors[i].next();
                         if (((ILSMTreeTupleReference) btreeCursors[i].getTuple()).isAntimatter()) {
                             searchCallback.cancel(predicate.getLowKey());
-                            btreeCursors[i].destroy();
+                            btreeCursors[i].close();
                             return false;
                         } else {
                             frameTuple = btreeCursors[i].getTuple();
@@ -120,7 +120,7 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
                         }
                     } else {
                         searchCallback.cancel(predicate.getLowKey());
-                        btreeCursors[i].destroy();
+                        btreeCursors[i].close();
                     }
                 } else {
                     frameTuple = btreeCursors[i].getTuple();
@@ -131,20 +131,16 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
                     return true;
                 }
             } else {
-                btreeCursors[i].destroy();
+                btreeCursors[i].close();
             }
         }
         return false;
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         try {
-            if (btreeCursors != null) {
-                for (int i = 0; i < numBTrees; ++i) {
-                    btreeCursors[i].close();
-                }
-            }
+            closeCursors();
             nextHasBeenCalled = false;
             foundTuple = false;
         } finally {
@@ -155,7 +151,7 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
         operationalComponents = lsmInitialState.getOperationalComponents();
         lsmHarness = lsmInitialState.getLSMHarness();
@@ -194,26 +190,23 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         nextHasBeenCalled = true;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        if (lsmHarness != null) {
-            try {
-                closeCursors();
-                btreeCursors = null;
-            } finally {
-                lsmHarness.endSearch(opCtx);
+    public void doDestroy() throws HyracksDataException {
+        if (btreeCursors != null) {
+            for (int i = 0; i < numBTrees; ++i) {
+                if (btreeCursors[i] != null) {
+                    btreeCursors[i].destroy();
+                }
             }
         }
-        nextHasBeenCalled = false;
-        foundTuple = false;
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 
@@ -240,7 +233,7 @@ public class LSMBTreePointSearchCursor implements ILSMIndexCursor {
         if (btreeCursors != null) {
             for (int i = 0; i < numBTrees; ++i) {
                 if (btreeCursors[i] != null) {
-                    btreeCursors[i].destroy();
+                    btreeCursors[i].close();
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 1e401a2..5d23fef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.PriorityQueue;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
@@ -64,13 +65,13 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
-        super.close();
+    public void doClose() throws HyracksDataException {
+        super.doClose();
         canCallProceed = true;
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         outputElement = outputPriorityQueue.poll();
         needPushElementIntoQueue = true;
         canCallProceed = false;
@@ -267,7 +268,6 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
                     TupleUtils.copyTuple(switchComponentTupleBuilders[i], element.getTuple(), cmp.getKeyFieldCount());
                 }
                 rangeCursors[i].close();
-                rangeCursors[i].destroy();
                 switchRequest[i] = true;
                 switchedElements[i] = element;
             }
@@ -318,7 +318,7 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
         cmp = lsmInitialState.getOriginalKeyComparator();
         operationalComponents = lsmInitialState.getOperationalComponents();
@@ -331,10 +331,19 @@ public class LSMBTreeRangeSearchCursor extends LSMIndexSearchCursor {
         includeMutableComponent = false;
 
         int numBTrees = operationalComponents.size();
-        if (rangeCursors == null || rangeCursors.length != numBTrees) {
+        if (rangeCursors == null) {
             // object creation: should be relatively low
             rangeCursors = new IIndexCursor[numBTrees];
             btreeAccessors = new BTreeAccessor[numBTrees];
+        } else if (rangeCursors.length != numBTrees) {
+            // should destroy first
+            Throwable failure = DestroyUtils.destroy(null, btreeAccessors);
+            failure = DestroyUtils.destroy(failure, rangeCursors);
+            if (failure != null) {
+                throw HyracksDataException.create(failure);
+            }
+            rangeCursors = new IIndexCursor[numBTrees];
+            btreeAccessors = new BTreeAccessor[numBTrees];
         }
 
         for (int i = 0; i < numBTrees; i++) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index 0ceaf7b..02574ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -24,11 +24,11 @@ import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 
-public class LSMBTreeSearchCursor implements ILSMIndexCursor {
+public class LSMBTreeSearchCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
 
     public enum LSMBTreeSearchType {
         POINT,
@@ -47,40 +47,43 @@ public class LSMBTreeSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
         RangePredicate btreePred = (RangePredicate) searchPred;
-
-        currentCursor =
-                btreePred.isPointPredicate(lsmInitialState.getOriginalKeyComparator()) ? pointCursor : rangeCursor;
+        currentCursor = lsmInitialState.isDiskComponentScan() ? scanCursor
+                : btreePred.isPointPredicate(lsmInitialState.getOriginalKeyComparator()) ? pointCursor : rangeCursor;
         currentCursor.open(lsmInitialState, searchPred);
     }
 
-    public void scan(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        currentCursor = scanCursor;
-        currentCursor.open(initialState, searchPred);
-    }
-
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         return currentCursor.hasNext();
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         currentCursor.next();
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        if (currentCursor != null) {
-            currentCursor.destroy();
+    public void doDestroy() throws HyracksDataException {
+        try {
+            pointCursor.destroy();
+        } finally {
+            try {
+                rangeCursor.destroy();
+            } finally {
+                try {
+                    scanCursor.destroy();
+                } finally {
+                    currentCursor = null;
+                }
+            }
         }
-        currentCursor = null;
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         if (currentCursor != null) {
             currentCursor.close();
         }
@@ -88,7 +91,7 @@ public class LSMBTreeSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return currentCursor.getTuple();
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
index b37fb20..8dcbcc4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyAbstractCursor.java
@@ -36,12 +36,12 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
 
-public abstract class LSMBTreeWithBuddyAbstractCursor implements ILSMIndexCursor {
+public abstract class LSMBTreeWithBuddyAbstractCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
 
     protected boolean open;
     protected BTreeRangeSearchCursor[] btreeCursors;
@@ -75,7 +75,7 @@ public abstract class LSMBTreeWithBuddyAbstractCursor implements ILSMIndexCursor
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
 
         LSMBTreeWithBuddyCursorInitialState lsmInitialState = (LSMBTreeWithBuddyCursorInitialState) initialState;
         btreeCmp = lsmInitialState.getBTreeCmp();
@@ -142,7 +142,7 @@ public abstract class LSMBTreeWithBuddyAbstractCursor implements ILSMIndexCursor
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         if (!open) {
             return;
         }
@@ -163,7 +163,7 @@ public abstract class LSMBTreeWithBuddyAbstractCursor implements ILSMIndexCursor
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
index c7f5cea..c66cd69 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
@@ -37,13 +37,13 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        super.destroy();
+    public void doDestroy() throws HyracksDataException {
+        super.doDestroy();
         currentCursor = 0;
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         if (!open) {
             return;
         }
@@ -52,8 +52,8 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
         foundNext = false;
         try {
             for (int i = 0; i < numberOfTrees; i++) {
-                btreeCursors[i].destroy();
-                buddyBtreeCursors[i].destroy();
+                btreeCursors[i].close();
+                buddyBtreeCursors[i].close();
             }
             btreeCursors = null;
             buddyBtreeCursors = null;
@@ -70,7 +70,7 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (foundNext) {
             return true;
         }
@@ -94,7 +94,7 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
                             killerTupleFound = true;
                         }
                     } finally {
-                        buddyBtreeCursors[i].destroy();
+                        buddyBtreeCursors[i].close();
                     }
                 }
                 if (!killerTupleFound) {
@@ -103,7 +103,7 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
                     return true;
                 }
             }
-            btreeCursors[currentCursor].destroy();
+            btreeCursors[currentCursor].close();
             currentCursor++;
             searchNextCursor();
         }
@@ -111,7 +111,7 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
     }
 
@@ -135,8 +135,8 @@ public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        super.open(initialState, searchPred);
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        super.doOpen(initialState, searchPred);
         searchNextCursor();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
index d889622..2913be1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
@@ -47,7 +47,7 @@ public class LSMBTreeWithBuddySortedCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         depletedBtreeCursors = new boolean[numberOfTrees];
         foundNext = false;
         try {
@@ -90,7 +90,7 @@ public class LSMBTreeWithBuddySortedCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         while (!foundNext) {
             frameTuple = null;
 
@@ -137,7 +137,7 @@ public class LSMBTreeWithBuddySortedCursor extends LSMBTreeWithBuddyAbstractCurs
                         break;
                     }
                 } finally {
-                    btreeCursors[i].destroy();
+                    btreeCursors[i].close();
                 }
             }
             if (!killed) {
@@ -149,14 +149,13 @@ public class LSMBTreeWithBuddySortedCursor extends LSMBTreeWithBuddyAbstractCurs
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         foundNext = false;
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        super.open(initialState, searchPred);
-
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        super.doOpen(initialState, searchPred);
         depletedBtreeCursors = new boolean[numberOfTrees];
         foundNext = false;
         for (int i = 0; i < numberOfTrees; i++) {
@@ -169,5 +168,4 @@ public class LSMBTreeWithBuddySortedCursor extends LSMBTreeWithBuddyAbstractCurs
             }
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
index b2f5327..fd13e62 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBuddyBTreeMergeCursor.java
@@ -45,7 +45,7 @@ public class LSMBuddyBTreeMergeCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeWithBuddyCursorInitialState lsmInitialState = (LSMBTreeWithBuddyCursorInitialState) initialState;
         cmp = lsmInitialState.getBuddyBTreeCmp();
         operationalComponents = lsmInitialState.getOperationalComponents();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
index b8d64af..0e1a5e4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -245,6 +245,9 @@ public interface ILSMIndexAccessor extends IIndexAccessor {
      * The returned tuples are first ordered on primary key, and then ordered on the descending order of
      * disk_component_position (older components get returned first)
      *
+     * If this method returns successfully, then the cursor has been opened. If an exception is thrown then
+     * the cursor was not opened
+     *
      * @param icursor
      *            Cursor over the index entries satisfying searchPred.
      * @throws HyracksDataException

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index ee7afa0..dcac219 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -213,4 +213,9 @@ public class ComponentReplacementContext implements ILSMIndexOperationContext {
     public boolean isTracingEnabled() {
         return false;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        // No Op.. Nothing to destroy
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 7f3a371..900ee32 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -32,10 +32,11 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.MultiComparator;
 
-public abstract class LSMIndexSearchCursor implements ILSMIndexCursor {
+public abstract class LSMIndexSearchCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
     protected static final int SWITCH_COMPONENT_CYCLE = 100;
     protected final ILSMIndexOperationContext opCtx;
     protected final boolean returnDeletedTuples;
@@ -106,7 +107,7 @@ public abstract class LSMIndexSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         hasNextCallCount = 0;
         switchPossible = true;
         outputElement = null;
@@ -133,20 +134,20 @@ public abstract class LSMIndexSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         hasNextCallCount++;
         checkPriorityQueue();
         return !outputPriorityQueue.isEmpty();
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         outputElement = outputPriorityQueue.poll();
         needPushElementIntoQueue = true;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         try {
             if (outputPriorityQueue != null) {
                 outputPriorityQueue.clear();
@@ -167,7 +168,7 @@ public abstract class LSMIndexSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return outputElement.getTuple();
     }
 
@@ -191,7 +192,7 @@ public abstract class LSMIndexSearchCursor implements ILSMIndexCursor {
             outputPriorityQueue.offer(e);
             return;
         }
-        rangeCursors[cursorIndex].destroy();
+        rangeCursors[cursorIndex].close();
         if (cursorIndex == 0) {
             includeMutableComponent = false;
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 4770d7c..9794a98 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -50,6 +50,7 @@ public class LSMTreeIndexAccessor implements ILSMIndexAccessor {
     protected final ILSMHarness lsmHarness;
     protected final ILSMIndexOperationContext ctx;
     protected final ICursorFactory cursorFactory;
+    private boolean destroyed = false;
 
     public LSMTreeIndexAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx, ICursorFactory cursorFactory) {
         this.lsmHarness = lsmHarness;
@@ -234,4 +235,13 @@ public class LSMTreeIndexAccessor implements ILSMIndexAccessor {
     public ILSMIndexOperationContext getOpContext() {
         return ctx;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        ctx.destroy();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index 1cfe345..ef9852d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -385,6 +385,11 @@ public class VirtualBufferCache implements IVirtualBufferCache {
         public CacheBucket() {
             this.bucketLock = new ReentrantLock();
         }
+
+        @Override
+        public String toString() {
+            return CacheBucket.class.getSimpleName() + " -> " + (cachedPage == null ? "" : cachedPage.toString());
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
index abea6a0..e9b3f21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/api/IInvertedListCursor.java
@@ -38,7 +38,7 @@ public interface IInvertedListCursor extends Comparable<IInvertedListCursor> {
     public ITupleReference getTuple();
 
     // getters
-    public int size();
+    public int size() throws HyracksDataException;
 
     public int getStartPageId();
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 05561f2..1beff71 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -270,28 +270,29 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
     @Override
     public ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException {
         LSMInvertedIndexFlushOperation flushOp = (LSMInvertedIndexFlushOperation) operation;
-
         // Create an inverted index instance to be bulk loaded.
         ILSMDiskComponent component = createDiskComponent(componentFactory, flushOp.getTarget(),
                 flushOp.getDeletedKeysBTreeTarget(), flushOp.getBloomFilterTarget(), true);
-
         // Create a scan cursor on the BTree underlying the in-memory inverted index.
         LSMInvertedIndexMemoryComponent flushingComponent =
                 (LSMInvertedIndexMemoryComponent) flushOp.getFlushingComponent();
-
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
-
         // Search the deleted keys BTree to calculate the number of elements for BloomFilter
+        long numBTreeTuples = 0L;
         BTreeAccessor deletedKeysBTreeAccessor =
                 flushingComponent.getBuddyIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
         IIndexCursor btreeCountingCursor = deletedKeysBTreeAccessor.createCountingSearchCursor();
-        deletedKeysBTreeAccessor.search(btreeCountingCursor, nullPred);
-        long numBTreeTuples = 0L;
         try {
-            while (btreeCountingCursor.hasNext()) {
-                btreeCountingCursor.next();
-                ITupleReference countTuple = btreeCountingCursor.getTuple();
-                numBTreeTuples = IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0));
+            deletedKeysBTreeAccessor.search(btreeCountingCursor, nullPred);
+            try {
+                while (btreeCountingCursor.hasNext()) {
+                    btreeCountingCursor.next();
+                    ITupleReference countTuple = btreeCountingCursor.getTuple();
+                    numBTreeTuples =
+                            IntegerPointable.getInteger(countTuple.getFieldData(0), countTuple.getFieldStart(0));
+                }
+            } finally {
+                btreeCountingCursor.close();
             }
         } finally {
             btreeCountingCursor.destroy();
@@ -302,29 +303,34 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
 
         // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
         IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
-        deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred);
-
         try {
-            while (deletedKeysScanCursor.hasNext()) {
-                deletedKeysScanCursor.next();
-                componentBulkLoader.delete(deletedKeysScanCursor.getTuple());
+            deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred);
+            try {
+                while (deletedKeysScanCursor.hasNext()) {
+                    deletedKeysScanCursor.next();
+                    componentBulkLoader.delete(deletedKeysScanCursor.getTuple());
+                }
+            } finally {
+                deletedKeysScanCursor.close();
             }
         } finally {
             deletedKeysScanCursor.destroy();
         }
-
         // Scan the in-memory inverted index
         InMemoryInvertedIndexAccessor memInvIndexAccessor =
                 flushingComponent.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
         BTreeAccessor memBTreeAccessor = memInvIndexAccessor.getBTreeAccessor();
         IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor(false);
-        memBTreeAccessor.search(scanCursor, nullPred);
-
-        // Bulk load the disk inverted index from the in-memory inverted index.
         try {
-            while (scanCursor.hasNext()) {
-                scanCursor.next();
-                componentBulkLoader.add(scanCursor.getTuple());
+            memBTreeAccessor.search(scanCursor, nullPred);
+            // Bulk load the disk inverted index from the in-memory inverted index.
+            try {
+                while (scanCursor.hasNext()) {
+                    scanCursor.next();
+                    componentBulkLoader.add(scanCursor.getTuple());
+                }
+            } finally {
+                scanCursor.close();
             }
         } finally {
             scanCursor.destroy();
@@ -337,86 +343,88 @@ public class LSMInvertedIndex extends AbstractLSMIndex implements IInvertedIndex
             filterManager.writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
         }
         flushingComponent.getMetadata().copy(component.getMetadata());
-
         componentBulkLoader.end();
-
         return component;
     }
 
     @Override
     public ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException {
         LSMInvertedIndexMergeOperation mergeOp = (LSMInvertedIndexMergeOperation) operation;
-        IIndexCursor cursor = mergeOp.getCursor();
-
         RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
-        ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
-        // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
-        search(opCtx, cursor, mergePred);
-
-        // Create an inverted index instance.
-        ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
-                mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
-
-        ILSMDiskComponentBulkLoader componentBulkLoader;
-
-        // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted indexes so that
-        // lsmHarness.endSearch() is called once when the inverted indexes have been merged.
-        if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
-                .get(diskComponents.size() - 1)) {
-            // Keep the deleted tuples since the oldest disk component is not included in the merge operation
-
-            LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
-                    new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
-            search(opCtx, btreeCursor, mergePred);
-
-            long numElements = 0L;
-            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i)).getBloomFilter()
-                        .getNumElements();
+        IIndexCursor cursor = mergeOp.getCursor();
+        try {
+            ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
+            // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
+            // Create an inverted index instance.
+            ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
+                    mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
+            ILSMDiskComponentBulkLoader componentBulkLoader;
+            // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
+            // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
+            if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
+                    .get(diskComponents.size() - 1)) {
+                // Keep the deleted tuples since the oldest disk component is not included in the merge operation
+                LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
+                        new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
+                try {
+                    long numElements = 0L;
+                    for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                        numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
+                                .getBloomFilter().getNumElements();
+                    }
+                    componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
+                    loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
+                } finally {
+                    btreeCursor.destroy();
+                }
+            } else {
+                componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
             }
-
-            componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
+            search(opCtx, cursor, mergePred);
             try {
-                while (btreeCursor.hasNext()) {
-                    btreeCursor.next();
-                    ITupleReference tuple = btreeCursor.getTuple();
-                    componentBulkLoader.delete(tuple);
+                while (cursor.hasNext()) {
+                    cursor.next();
+                    ITupleReference tuple = cursor.getTuple();
+                    componentBulkLoader.add(tuple);
                 }
             } finally {
-                btreeCursor.destroy();
+                cursor.close();
             }
-        } else {
-            componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
+            if (component.getLSMComponentFilter() != null) {
+                List<ITupleReference> filterTuples = new ArrayList<>();
+                for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+                    ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
+                    ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
+                    if (min != null) {
+                        filterTuples.add(min);
+                    }
+                    if (max != null) {
+                        filterTuples.add(max);
+                    }
+                }
+                getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
+                getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+            }
+            componentBulkLoader.end();
+            return component;
+        } finally {
+            cursor.close();
         }
+    }
 
+    private void loadDeleteTuples(ILSMIndexOperationContext opCtx,
+            LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor, RangePredicate mergePred,
+            ILSMDiskComponentBulkLoader componentBulkLoader) throws HyracksDataException {
+        search(opCtx, btreeCursor, mergePred);
         try {
-            while (cursor.hasNext()) {
-                cursor.next();
-                ITupleReference tuple = cursor.getTuple();
-                componentBulkLoader.add(tuple);
+            while (btreeCursor.hasNext()) {
+                btreeCursor.next();
+                ITupleReference tuple = btreeCursor.getTuple();
+                componentBulkLoader.delete(tuple);
             }
         } finally {
-            cursor.destroy();
+            btreeCursor.close();
         }
-        if (component.getLSMComponentFilter() != null) {
-            List<ITupleReference> filterTuples = new ArrayList<>();
-            for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
-                ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
-                ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
-                if (min != null) {
-                    filterTuples.add(min);
-                }
-                if (max != null) {
-                    filterTuples.add(max);
-                }
-            }
-            getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples);
-            getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
-        }
-
-        componentBulkLoader.end();
-
-        return component;
     }
 
     protected InMemoryInvertedIndex createInMemoryInvertedIndex(IVirtualBufferCache virtualBufferCache,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 61fc84e..c33e2ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -43,6 +43,7 @@ public class LSMInvertedIndexAccessor implements ILSMIndexAccessor, IInvertedInd
 
     protected final ILSMHarness lsmHarness;
     protected final ILSMIndexOperationContext ctx;
+    private boolean destroyed = false;
 
     public LSMInvertedIndexAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) {
         this.lsmHarness = lsmHarness;
@@ -225,4 +226,13 @@ public class LSMInvertedIndexAccessor implements ILSMIndexAccessor, IInvertedInd
     public ILSMIndexOperationContext getOpContext() {
         return ctx;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        ctx.destroy();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
index 2a105f1..21ce940 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
@@ -43,7 +43,7 @@ public class LSMInvertedIndexDeletedKeysBTreeMergeCursor extends LSMIndexSearchC
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMInvertedIndexRangeSearchCursorInitialState lsmInitialState =
                 (LSMInvertedIndexRangeSearchCursorInitialState) initialState;
         cmp = lsmInitialState.getOriginalKeyComparator();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
index 55da252..cf95f78 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
 import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
 import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
@@ -46,6 +47,7 @@ public class LSMInvertedIndexOpContext extends AbstractLSMIndexOperationContext
     private IIndexAccessor[] deletedKeysBTreeAccessors;
     private IInvertedIndexAccessor currentMutableInvIndexAccessors;
     private IIndexAccessor currentDeletedKeysBTreeAccessors;
+    private boolean destroyed = false;
 
     public LSMInvertedIndexOpContext(ILSMIndex index, List<ILSMMemoryComponent> mutableComponents,
             IModificationOperationCallback modificationCallback, ISearchOperationCallback searchCallback,
@@ -94,4 +96,17 @@ public class LSMInvertedIndexOpContext extends AbstractLSMIndexOperationContext
     public IIndexAccessor getCurrentDeletedKeysBTreeAccessors() {
         return currentDeletedKeysBTreeAccessors;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        Throwable failure = DestroyUtils.destroy(null, mutableInvIndexAccessors);
+        failure = DestroyUtils.destroy(failure, deletedKeysBTreeAccessors);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
index 4afccef..12dc23f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -51,12 +51,7 @@ public class LSMInvertedIndexRangeSearchCursor extends LSMIndexSearchCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
-        super.next();
-    }
-
-    @Override
-    public void open(ICursorInitialState initState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMInvertedIndexRangeSearchCursorInitialState lsmInitState =
                 (LSMInvertedIndexRangeSearchCursorInitialState) initState;
         cmp = lsmInitState.getOriginalKeyComparator();
@@ -107,13 +102,13 @@ public class LSMInvertedIndexRangeSearchCursor extends LSMIndexSearchCursor {
                 continue;
             }
             deletedKeysBTreeCursors[i].close();
+            deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
             try {
-                deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
                 if (deletedKeysBTreeCursors[i].hasNext()) {
                     return true;
                 }
             } finally {
-                deletedKeysBTreeCursors[i].destroy();
+                deletedKeysBTreeCursors[i].close();
             }
         }
         return false;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index ce9f760..fea9373 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -30,6 +30,7 @@ import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentTy
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -41,7 +42,7 @@ import org.apache.hyracks.storage.common.MultiComparator;
  * Searches the components one-by-one, completely consuming a cursor before moving on to the next one.
  * Therefore, the are no guarantees about sort order of the results.
  */
-public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
+public class LSMInvertedIndexSearchCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
 
     private IIndexAccessor currentAccessor;
     private IIndexCursor currentCursor;
@@ -66,7 +67,7 @@ public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
     private final long[] hashes = BloomFilter.createHashArray();
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMInvertedIndexSearchCursorInitialState lsmInitState = (LSMInvertedIndexSearchCursorInitialState) initialState;
         harness = lsmInitState.getLSMHarness();
         operationalComponents = lsmInitState.getOperationalComponents();
@@ -109,7 +110,7 @@ public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
                     return true;
                 }
             } finally {
-                deletedKeysBTreeCursors[i].destroy();
+                deletedKeysBTreeCursors[i].close();
             }
         }
         return false;
@@ -140,7 +141,7 @@ public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (!tupleConsumed) {
             return true;
         }
@@ -148,7 +149,7 @@ public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
             if (nextValidTuple()) {
                 return true;
             }
-            currentCursor.destroy();
+            currentCursor.close();
             accessorIndex++;
         }
         while (accessorIndex < indexAccessors.size()) {
@@ -160,28 +161,28 @@ public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
                 return true;
             }
             // Close as we go to release resources.
-            currentCursor.destroy();
+            currentCursor.close();
             accessorIndex++;
         }
         return false;
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         // Mark the tuple as consumed, so hasNext() can move on.
         tupleConsumed = true;
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        close();
+    public void doDestroy() throws HyracksDataException {
+        doClose();
     }
 
     @Override
-    public void close() throws HyracksDataException {
+    public void doClose() throws HyracksDataException {
         try {
             if (currentCursor != null) {
-                currentCursor.destroy();
+                currentCursor.close();
                 currentCursor = null;
             }
             accessorIndex = 0;
@@ -193,7 +194,7 @@ public class LSMInvertedIndexSearchCursor implements ILSMIndexCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return currentCursor.getTuple();
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index 6777177..0795a4e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -46,6 +46,7 @@ public class InMemoryInvertedIndexAccessor implements IInvertedIndexAccessor {
     protected IIndexOperationContext opCtx;
     protected InMemoryInvertedIndex index;
     protected BTreeAccessor btreeAccessor;
+    private boolean destroyed = false;
 
     public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex index, IIndexOperationContext opCtx)
             throws HyracksDataException {
@@ -130,4 +131,20 @@ public class InMemoryInvertedIndexAccessor implements IInvertedIndexAccessor {
         btreeAccessor.getOpContext().resetNonIndexFieldsTuple(newTuple);
     }
 
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        doDestroy();
+    }
+
+    private void doDestroy() throws HyracksDataException {
+        try {
+            btreeAccessor.destroy();
+        } finally {
+            opCtx.destroy();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
index eec20fd..0457b46 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.lsm.invertedindex.inmemory;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
@@ -46,6 +47,7 @@ public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
     // To generate in-memory BTree tuples for insertions.
     private final IBinaryTokenizerFactory tokenizerFactory;
     private InvertedIndexTokenizingTupleIterator tupleIter;
+    private boolean destroyed = false;
 
     InMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory) {
@@ -123,4 +125,15 @@ public class InMemoryInvertedIndexOpContext implements IIndexOperationContext {
     public void setTupleIter(InvertedIndexTokenizingTupleIterator tupleIter) {
         this.tupleIter = tupleIter;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        if (btreeAccessor != null) {
+            btreeAccessor.destroy();
+        }
+    }
 }


[6/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
deleted file mode 100644
index 4d0f287..0000000
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.hyracks.tests.unit;
-
-import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.List;
-
-/**
- * This is a test class that forms the basis for unit tests of different implementations of the IIndexCursor interface
- */
-public abstract class IIndexCursorTest {
-    @Test
-    public void testNormalLifeCycle() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        for (ISearchPredicate predicate : predicates) {
-            cursor.open(initialState, predicate);
-            while (cursor.hasNext()) {
-                cursor.next();
-            }
-            cursor.close();
-        }
-        cursor.destroy();
-    }
-
-    @Test
-    public void testCreateDestroySucceed() throws Exception {
-        IIndexCursor cursor = createCursor();
-        cursor.destroy();
-    }
-
-    @Test
-    public void testDoubleOpenFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.open(initialState, predicates.get(0));
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.close();
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testCloseWithoutOpenFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.close();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testDoubleCloseFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.close();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testHasNextBeforeOpenFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.hasNext();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testHasNextAfterCloseFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.hasNext();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testNextBeforeOpenFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.next();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testNextAfterCloseFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.next();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testDestroyWhileOpenFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.destroy();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        cursor.close();
-        cursor.destroy();
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testOpenAfterDestroyFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        cursor.destroy();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.open(initialState, predicates.get(0));
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testCloseAfterDestroyFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        cursor.destroy();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.close();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testNextAfterDestroyFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        cursor.destroy();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.next();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testHasNextAfterDestroyFails() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        cursor.destroy();
-        boolean expectedExceptionThrown = false;
-        try {
-            cursor.hasNext();
-        } catch (Exception e) {
-            expectedExceptionThrown = true;
-        }
-        Assert.assertTrue(expectedExceptionThrown);
-    }
-
-    @Test
-    public void testGetTupleReturnsNullAfterDestroy() throws Exception {
-        IIndexCursor cursor = createCursor();
-        ICursorInitialState initialState = createCursorInitialState();
-        List<ISearchPredicate> predicates = createSearchPredicates();
-        cursor.open(initialState, predicates.get(0));
-        cursor.close();
-        cursor.destroy();
-        Assert.assertNull(cursor.getTuple());
-    }
-
-    protected abstract List<ISearchPredicate> createSearchPredicates();
-
-    protected abstract ICursorInitialState createCursorInitialState();
-
-    protected abstract IIndexCursor createCursor();
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index 3c045cb..a296672 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -29,10 +29,10 @@ import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.api.ITupleUpdater;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.common.IIndexCursor;
 
 public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
     private final ITupleUpdater tupleUpdater;
@@ -49,7 +49,7 @@ public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNo
     }
 
     @Override
-    protected ITreeIndexCursor createCursor() {
+    protected IIndexCursor createCursor() {
         ITreeIndex treeIndex = (ITreeIndex) index;
         ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
         return new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 6e2d694..077a006 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -841,6 +841,7 @@ public class BTree extends AbstractTreeIndex {
     public class BTreeAccessor implements ITreeIndexAccessor {
         protected BTree btree;
         protected BTreeOpContext ctx;
+        private boolean destroyed = false;
 
         public BTreeAccessor(BTree btree, IModificationOperationCallback modificationCalback,
                 ISearchOperationCallback searchCallback) {
@@ -995,6 +996,15 @@ public class BTree extends AbstractTreeIndex {
                 throw HyracksDataException.create(ErrorCode.OPERATION_EXCEEDED_MAX_RESTARTS, MAX_RESTARTS);
             }
         }
+
+        @Override
+        public void destroy() throws HyracksDataException {
+            if (destroyed) {
+                return;
+            }
+            destroyed = true;
+            ctx.destroy();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
index 10f79a2..f7e0ce0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
@@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -36,7 +37,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
-public class BTreeCountingSearchCursor implements ITreeIndexCursor {
+public class BTreeCountingSearchCursor extends EnforcedIndexCursor implements ITreeIndexCursor {
 
     private int fileId = -1;
     private ICachedPage page = null;
@@ -75,11 +76,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        // in case open is called multiple times without closing
-        if (page != null) {
-            releasePage();
-        }
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
 
         page = ((BTreeCursorInitialState) initialState).getPage();
         isPageDirty = false;
@@ -169,7 +166,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         // get the count for the current page
         // follow the sibling pointer until last page
         // if no more tuples on a page, then done
@@ -199,7 +196,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         // Do nothing. Count is performed just once!
         IntegerPointable.setInteger(countBuf, 0, count);
         tupleBuilder.addField(countBuf, 0, 4);
@@ -207,7 +204,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         if (page != null) {
             releasePage();
         }
@@ -220,16 +217,12 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void close() {
-        try {
-            destroy();
-        } catch (Exception e) {
-            e.printStackTrace();
-        }
+    public void doClose() throws HyracksDataException {
+        doDestroy();
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return countTuple;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
index a46ae9a..60fa145 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
@@ -24,6 +24,7 @@ import java.util.Deque;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -73,6 +74,7 @@ public class BTreeOpContext implements IIndexOperationContext, IExtraPageBlockHe
     private ISearchOperationCallback searchCallback;
     private ITupleAcceptor acceptor;
     private int smoCount;
+    private boolean destroyed = false;
 
     // Debug
     private final Deque<PageValidationInfo> validationInfos;
@@ -383,4 +385,16 @@ public class BTreeOpContext implements IIndexOperationContext, IExtraPageBlockHe
     public void resetNonIndexFieldsTuple(ITupleReference newValue) {
         tupleWithNonIndexFields.reset(newValue);
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        Throwable failure = DestroyUtils.destroy(null, accessor, cursor);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index e903180..bff1bcb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -29,6 +29,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import org.apache.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.IIndexAccessor;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -38,7 +39,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
-public class BTreeRangeSearchCursor implements ITreeIndexCursor {
+public class BTreeRangeSearchCursor extends EnforcedIndexCursor implements ITreeIndexCursor {
 
     protected final IBTreeLeafFrame frame;
     protected final ITreeIndexTupleReference frameTuple;
@@ -81,19 +82,12 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
-        if (page != null) {
-            releasePage();
-        }
-
-        tupleIndex = 0;
-        page = null;
-        isPageDirty = false;
-        pred = null;
+    public void doDestroy() throws HyracksDataException {
+        // No Op all resources are released in the close call
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 
@@ -118,7 +112,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         int nextLeafPage;
         if (tupleIndex >= frame.getTupleCount()) {
             nextLeafPage = frame.getNextLeaf();
@@ -159,8 +153,10 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor {
 
                 // retraverse the index looking for the reconciled key
                 reusablePredicate.setLowKey(reconciliationTuple, true);
+                // before re-using the cursor, we must close it
+                close();
+                // this search call will re-open the cursor
                 accessor.search(this, reusablePredicate);
-
                 if (stopTupleIndex < 0 || tupleIndex > stopTupleIndex) {
                     return false;
                 }
@@ -177,7 +173,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         tupleIndex++;
     }
 
@@ -216,7 +212,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (page != null) {
             resetBeforeOpen();
@@ -262,8 +258,15 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
-        destroy();
+    public void doClose() throws HyracksDataException {
+        if (page != null) {
+            releasePage();
+        }
+
+        tupleIndex = 0;
+        page = null;
+        isPageDirty = false;
+        pred = null;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
index 32fa9df..eee43b5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTree.java
@@ -275,7 +275,10 @@ public class DiskBTree extends BTree {
 
         @Override
         protected void releasePage() throws HyracksDataException {
-            bufferCache.unpin(page);
+            if (page != null) {
+                bufferCache.unpin(page);
+                page = null;
+            }
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
index 5839d0e..7814e60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreePointSearchCursor.java
@@ -35,17 +35,17 @@ public class DiskBTreePointSearchCursor extends DiskBTreeRangeSearchCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         return tupleIndex >= 0 && !nextHasBeenCalled;
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         nextHasBeenCalled = true;
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (page != null) {
             resetBeforeOpen();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java
index 7d4ee0d..0e82088 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/DiskBTreeRangeSearchCursor.java
@@ -37,7 +37,7 @@ public class DiskBTreeRangeSearchCursor extends BTreeRangeSearchCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         int nextLeafPage;
         if (tupleIndex >= frame.getTupleCount()) {
             nextLeafPage = frame.getNextLeaf();
@@ -99,8 +99,8 @@ public class DiskBTreeRangeSearchCursor extends BTreeRangeSearchCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
-        super.close();
+    public void doClose() throws HyracksDataException {
+        super.doClose();
         searchPages.clear();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 4ff0656..12dc310 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -67,10 +67,12 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
 @RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
 @PrepareForTest({ BTreeUtils.class, FrameTupleAccessor.class, ArrayTupleBuilder.class,
         IndexSearchOperatorNodePushable.class, FrameUtils.class, FrameTupleAppender.class })
 public class FramewriterTest {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
index 94fe2c2..5f05ce9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -16,18 +16,16 @@
  ! specific language governing permissions and limitations
  ! under the License.
  !-->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hyracks-storage-am-common</artifactId>
   <name>hyracks-storage-am-common</name>
-
   <parent>
     <groupId>org.apache.hyracks</groupId>
     <artifactId>hyracks</artifactId>
     <version>0.3.4-SNAPSHOT</version>
   </parent>
-
   <licenses>
     <license>
       <name>Apache License, Version 2.0</name>
@@ -36,11 +34,24 @@
       <comments>A business-friendly OSS license</comments>
     </license>
   </licenses>
-
   <properties>
     <root.dir>${basedir}/../..</root.dir>
   </properties>
-
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
   <dependencies>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -94,5 +105,10 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java
index a8bdf32..01c98f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexOperationContext.java
@@ -18,13 +18,26 @@
  */
 package org.apache.hyracks.storage.am.common.api;
 
+import org.apache.hyracks.api.dataflow.IDestroyable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 
-public interface IIndexOperationContext {
+public interface IIndexOperationContext extends IDestroyable {
+    /**
+     * Set the next intended operation for this context
+     *
+     * @param newOp
+     * @throws HyracksDataException
+     */
     void setOperation(IndexOperation newOp) throws HyracksDataException;
 
+    /**
+     * @return the intended operation
+     */
     IndexOperation getOperation();
 
+    /**
+     * Release resources associated with the context and prepare it for the next use
+     */
     void reset();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java
index 90963bf..0d7fd5f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexAccessor.java
@@ -39,6 +39,8 @@ public interface ITreeIndexAccessor extends IIndexAccessor {
     /**
      * Open the given cursor for a disk-order scan, positioning the cursor to
      * the first leaf tuple.
+     * If this method returns successfully, the cursor is open.
+     * Otherwise, it was not open
      *
      * @param icursor
      *            Cursor to be opened for disk-order scanning.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
index aae830d..8490c6a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
@@ -21,7 +21,6 @@ package org.apache.hyracks.storage.am.common.dataflow;
 
 import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_DROP_IN_USE_INDEX;
 import static org.apache.hyracks.api.exceptions.ErrorCode.INDEX_DOES_NOT_EXIST;
-import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
 import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption.IF_EXISTS;
 import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption.WAIT_ON_IN_USE;
 
@@ -34,6 +33,7 @@ import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -109,7 +109,6 @@ public class IndexDropOperatorNodePushable extends AbstractOperatorNodePushable
                 maxWaitTimeMillis -= DROP_ATTEMPT_WAIT_TIME_MILLIS;
                 return true;
             } catch (InterruptedException e1) {
-                Thread.currentThread().interrupt();
                 throw HyracksDataException.create(e1);
             }
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index d55962a..41fdc41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -29,6 +29,8 @@ import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.profiling.IOperatorStats;
+import org.apache.hyracks.api.util.DestroyUtils;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.common.job.profiling.OperatorStats;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -43,6 +45,7 @@ import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
+import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexAccessor;
@@ -218,7 +221,15 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
 
     @Override
     public void close() throws HyracksDataException {
-        HyracksDataException closeException = null;
+        Throwable failure = releaseResources();
+        failure = ResourceReleaseUtils.close(writer, failure);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
+    }
+
+    private Throwable releaseResources() {
+        Throwable failure = null;
         if (index != null) {
             // if index == null, then the index open was not successful
             if (!failed) {
@@ -226,44 +237,24 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
                     if (appender.getTupleCount() > 0) {
                         appender.write(writer, true);
                     }
-                } catch (Throwable th) {
-                    writer.fail();
-                    closeException = HyracksDataException.create(th);
-                }
-            }
-
-            try {
-                cursor.destroy();
-            } catch (Throwable th) {
-                if (closeException == null) {
-                    closeException = HyracksDataException.create(th);
-                } else {
-                    closeException.addSuppressed(th);
+                } catch (Throwable th) { // NOSONAR Must ensure writer.fail is called.
+                    // subsequently, the failure will be thrown
+                    failure = th;
                 }
-            }
-            try {
-                indexHelper.close();
-            } catch (Throwable th) {
-                if (closeException == null) {
-                    closeException = new HyracksDataException(th);
-                } else {
-                    closeException.addSuppressed(th);
+                if (failure != null) {
+                    try {
+                        writer.fail();
+                    } catch (Throwable th) {// NOSONAR Must cursor.close is called.
+                        // subsequently, the failure will be thrown
+                        failure = ExceptionUtils.suppress(failure, th);
+                    }
                 }
             }
+            failure = ResourceReleaseUtils.close(cursor, failure);
+            failure = DestroyUtils.destroy(failure, cursor, indexAccessor);
+            failure = ResourceReleaseUtils.close(indexHelper, failure);
         }
-        try {
-            // will definitely be called regardless of exceptions
-            writer.close();
-        } catch (Throwable th) {
-            if (closeException == null) {
-                closeException = new HyracksDataException(th);
-            } else {
-                closeException.addSuppressed(th);
-            }
-        }
-        if (closeException != null) {
-            throw closeException;
-        }
+        return failure;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 7626815..90b50c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -19,10 +19,12 @@
 package org.apache.hyracks.storage.am.common.dataflow;
 
 import java.io.DataOutput;
+import java.io.IOException;
 
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -37,6 +39,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
+import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
 import org.apache.hyracks.storage.common.IIndexAccessParameters;
 import org.apache.hyracks.storage.common.ISearchOperationCallback;
 import org.apache.hyracks.storage.common.LocalResource;
@@ -56,55 +59,69 @@ public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOut
 
     @Override
     public void initialize() throws HyracksDataException {
+        Throwable failure = null;
         treeIndexHelper.open();
-        ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndexInstance();
         try {
-            ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
-            ITreeIndexCursor cursor = new TreeIndexDiskOrderScanCursor(cursorFrame);
-            LocalResource resource = treeIndexHelper.getResource();
-            ISearchOperationCallback searchCallback =
-                    searchCallbackFactory.createSearchOperationCallback(resource.getId(), ctx, null);
-            IIndexAccessParameters iap = new IndexAccessParameters(NoOpOperationCallback.INSTANCE, searchCallback);
-            ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex.createAccessor(iap);
+            writer.open();
+            FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
+            scan(appender);
+            appender.write(writer, true);
+        } catch (Throwable th) { // NOSONAR: Must call writer.fail
+            failure = th;
             try {
-                writer.open();
-                indexAccessor.diskOrderScan(cursor);
-                int fieldCount = treeIndex.getFieldCount();
-                FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
-                ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
-                DataOutput dos = tb.getDataOutput();
+                writer.fail();
+            } catch (Throwable failFailure) {// NOSONAR: Must maintain all stacks
+                failure = ExceptionUtils.suppress(failure, failFailure);
+            }
+        } finally {
+            failure = ResourceReleaseUtils.close(writer, failure);
+        }
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
+        }
+    }
+
+    private void scan(FrameTupleAppender appender) throws IOException {
+        ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndexInstance();
+        LocalResource resource = treeIndexHelper.getResource();
+        ISearchOperationCallback searchCallback =
+                searchCallbackFactory.createSearchOperationCallback(resource.getId(), ctx, null);
+        IIndexAccessParameters iap = new IndexAccessParameters(NoOpOperationCallback.INSTANCE, searchCallback);
+        ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex.createAccessor(iap);
+        try {
+            doScan(treeIndex, indexAccessor, appender);
+        } finally {
+            indexAccessor.destroy();
+        }
+    }
 
+    private void doScan(ITreeIndex treeIndex, ITreeIndexAccessor indexAccessor, FrameTupleAppender appender)
+            throws IOException {
+        int fieldCount = treeIndex.getFieldCount();
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        DataOutput dos = tb.getDataOutput();
+        ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
+        ITreeIndexCursor cursor = new TreeIndexDiskOrderScanCursor(cursorFrame);
+        try {
+            indexAccessor.diskOrderScan(cursor);
+            try {
                 while (cursor.hasNext()) {
                     tb.reset();
                     cursor.next();
-
                     ITupleReference frameTuple = cursor.getTuple();
                     for (int i = 0; i < frameTuple.getFieldCount(); i++) {
                         dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i),
                                 frameTuple.getFieldLength(i));
                         tb.addFieldEndOffset();
                     }
-
                     FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
                             tb.getSize());
-
                 }
-                appender.write(writer, true);
-            } catch (Throwable th) {
-                writer.fail();
-                throw new HyracksDataException(th);
             } finally {
-                try {
-                    cursor.destroy();
-                } catch (Exception cursorCloseException) {
-                    throw new IllegalStateException(cursorCloseException);
-                } finally {
-                    writer.close();
-                }
+                cursor.close();
             }
-        } catch (Throwable th) {
-            treeIndexHelper.close();
-            throw new HyracksDataException(th);
+        } finally {
+            cursor.destroy();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
index b2c26db..36fba76 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
@@ -20,17 +20,17 @@
 package org.apache.hyracks.storage.am.common.impls;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
-public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
+public class TreeIndexDiskOrderScanCursor extends EnforcedIndexCursor implements ITreeIndexCursor {
 
     protected int tupleIndex = 0;
     protected int fileId = -1;
@@ -48,13 +48,15 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
+        tupleIndex = 0;
+        currentPageId = -1;
+        maxPageId = -1;
         releasePage();
-        page = null;
     }
 
     @Override
-    public ITreeIndexTupleReference getTuple() {
+    public ITreeIndexTupleReference doGetTuple() {
         return frameTuple;
     }
 
@@ -65,7 +67,6 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
             }
 
             releasePage();
-
             ICachedPage nextPage = acquireNextPage();
             page = nextPage;
             frame.setPage(page);
@@ -80,7 +81,7 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (currentPageId > maxPageId) {
             return false;
         }
@@ -98,16 +99,12 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         tupleIndex++;
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        // in case open is called multiple times without closing
-        if (page != null) {
-            releasePage();
-        }
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         page = initialState.getPage();
         tupleIndex = 0;
         frame.setPage(page);
@@ -115,11 +112,11 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void close() {
+    public void doClose() throws HyracksDataException {
         tupleIndex = 0;
         currentPageId = -1;
         maxPageId = -1;
-        page = null;
+        releasePage();
     }
 
     @Override
@@ -146,8 +143,11 @@ public class TreeIndexDiskOrderScanCursor implements ITreeIndexCursor {
     }
 
     protected void releasePage() throws HyracksDataException {
-        page.releaseReadLatch();
-        bufferCache.unpin(page);
+        if (page != null) {
+            page.releaseReadLatch();
+            bufferCache.unpin(page);
+            page = null;
+        }
     }
 
     protected ICachedPage acquireNextPage() throws HyracksDataException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java
new file mode 100644
index 0000000..50b6e59
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/ResourceReleaseUtils.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.util;
+
+import java.util.List;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ResourceReleaseUtils {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private ResourceReleaseUtils() {
+    }
+
+    /**
+     * Close the cursor and suppress any Throwable thrown by the close call.
+     * This method must NEVER throw any Throwable
+     *
+     * @param cursor
+     *            the cursor to close
+     * @param root
+     *            the first exception encountered during release of resources
+     * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+     */
+    public static Throwable close(IIndexCursor cursor, Throwable root) {
+        if (cursor != null) {
+            try {
+                cursor.close();
+            } catch (Throwable th) { // NOSONAR Will be re-thrown
+                try {
+                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                } catch (Throwable loggingFailure) {
+                    // Do nothing
+                }
+                root = ExceptionUtils.suppress(root, th);
+            }
+        }
+        return root;
+    }
+
+    /**
+     * Close the AutoCloseable and suppress any Throwable thrown by the close call.
+     * This method must NEVER throw any Throwable
+     *
+     * @param closable
+     *            the resource to close
+     * @param root
+     *            the first exception encountered during release of resources
+     * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+     */
+    public static Throwable close(AutoCloseable closable, Throwable root) {
+        if (closable != null) {
+            try {
+                closable.close();
+            } catch (Throwable th) { // NOSONAR Will be re-thrown
+                try {
+                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                } catch (Throwable loggingFailure) {
+                    // Do nothing
+                }
+                root = ExceptionUtils.suppress(root, th);
+            }
+        }
+        return root;
+    }
+
+    /**
+     * Close the IIndexDataflowHelper and suppress any Throwable thrown by the close call.
+     * This method must NEVER throw any Throwable
+     *
+     * @param indexHelper
+     *            the indexHelper to close
+     * @param root
+     *            the first exception encountered during release of resources
+     * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+     */
+    public static Throwable close(IIndexDataflowHelper indexHelper, Throwable root) {
+        if (indexHelper != null) {
+            try {
+                indexHelper.close();
+            } catch (Throwable th) { // NOSONAR Will be re-thrown
+                try {
+                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                } catch (Throwable loggingFailure) {
+                    // Do nothing
+                }
+                root = ExceptionUtils.suppress(root, th);
+            }
+        }
+        return root;
+    }
+
+    /**
+     * Close the IIndexDataflowHelpers and suppress any Throwable thrown by any close call.
+     * This method must NEVER throw any Throwable
+     *
+     * @param indexHelpers
+     *            the indexHelpers to close
+     * @param root
+     *            the first exception encountered during release of resources
+     * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+     */
+    public static Throwable close(List<IIndexDataflowHelper> indexHelpers, Throwable root) {
+        for (int i = 0; i < indexHelpers.size(); i++) {
+            root = close(indexHelpers.get(i), root);
+        }
+        return root;
+    }
+
+    /**
+     * Close the IFrameWriter and suppress any Throwable thrown by the close call.
+     * This method must NEVER throw any Throwable
+     *
+     * @param writer
+     *            the writer to close
+     * @param root
+     *            the first exception encountered during release of resources
+     * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+     */
+    public static Throwable close(IFrameWriter writer, Throwable root) {
+        if (writer != null) {
+            try {
+                writer.close();
+            } catch (Throwable th) { // NOSONAR Will be re-thrown
+                try {
+                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                } catch (Throwable loggingFailure) {
+                    // Do nothing
+                }
+                root = ExceptionUtils.suppress(root, th);
+            }
+        }
+        return root;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java
new file mode 100644
index 0000000..eec9542
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/DummyEnforcedIndexCursor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.test;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+public class DummyEnforcedIndexCursor extends EnforcedIndexCursor {
+    @Override
+    protected void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // Dummy
+    }
+
+    @Override
+    protected boolean doHasNext() throws HyracksDataException {
+        // Dummy
+        return false;
+    }
+
+    @Override
+    protected void doNext() throws HyracksDataException {
+        // Dummy
+    }
+
+    @Override
+    protected void doDestroy() throws HyracksDataException {
+        // Dummy
+    }
+
+    @Override
+    protected void doClose() throws HyracksDataException {
+        // Dummy
+    }
+
+    @Override
+    protected ITupleReference doGetTuple() {
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java
new file mode 100644
index 0000000..9625c2f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/EnforcedIndexCursorTest.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.common.test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class EnforcedIndexCursorTest extends IIndexCursorTest {
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            predicates.add(Mockito.mock(ISearchPredicate.class));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws HyracksDataException {
+        EnforcedIndexCursor cursor = new DummyEnforcedIndexCursor();
+        IIndexAccessor accessor = Mockito.mock(IIndexAccessor.class);
+        Mockito.when(accessor.createSearchCursor(Mockito.anyBoolean())).thenReturn(cursor);
+        Mockito.doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                Object[] args = invocation.getArguments();
+                IIndexCursor cursor = (IIndexCursor) args[0];
+                ISearchPredicate pred = (ISearchPredicate) args[1];
+                cursor.open(null, pred);
+                return null;
+            }
+        }).when(accessor).search(Matchers.any(IIndexCursor.class), Matchers.any(ISearchPredicate.class));
+        return accessor;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java
new file mode 100644
index 0000000..6a3fdf1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/test/java/org/apache/hyracks/storage/am/common/test/IIndexCursorTest.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.storage.am.common.test;
+
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * This is a test class that forms the basis for unit tests of different implementations of the IIndexCursor interface
+ */
+public abstract class IIndexCursorTest {
+    @Test
+    public void testNormalLifeCycle() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        for (ISearchPredicate predicate : predicates) {
+            open(accessor, cursor, predicate);
+            while (cursor.hasNext()) {
+                cursor.next();
+            }
+            cursor.close();
+        }
+        cursor.destroy();
+        destroy(accessor);
+    }
+
+    protected void destroy(IIndexAccessor accessor) throws HyracksDataException {
+        accessor.destroy();
+    }
+
+    @Test
+    public void testCreateDestroySucceed() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        cursor.destroy();
+        destroy(accessor);
+    }
+
+    @Test
+    public void testDoubleOpenFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        boolean expectedExceptionThrown = false;
+        try {
+            open(accessor, cursor, predicates.get(0));
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.close();
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testCloseWithoutOpenSucceeds() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        cursor.close();
+        cursor.destroy();
+        destroy(accessor);
+    }
+
+    @Test
+    public void testDoubleCloseSucceeds() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.close();
+        cursor.destroy();
+        destroy(accessor);
+    }
+
+    @Test
+    public void testDoubleDestroySucceeds() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        cursor.destroy();
+        destroy(accessor);
+    }
+
+    @Test
+    public void testHasNextBeforeOpenFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextAfterCloseFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextBeforeOpenFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextAfterCloseFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testDestroyWhileOpenFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.destroy();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.close();
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testOpenAfterDestroyFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            open(accessor, cursor, predicates.get(0));
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testCloseAfterDestroyFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextAfterDestroyFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextAfterDestroyFails() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        destroy(accessor);
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testGetTupleReturnsNullAfterDestroy() throws Exception {
+        IIndexAccessor accessor = createAccessor();
+        IIndexCursor cursor = createCursor(accessor);
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        open(accessor, cursor, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        destroy(accessor);
+        Assert.assertNull(cursor.getTuple());
+    }
+
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        return accessor.createSearchCursor(false);
+    }
+
+    protected void open(IIndexAccessor accessor, IIndexCursor cursor, ISearchPredicate predicate)
+            throws HyracksDataException {
+        accessor.search(cursor, predicate);
+    }
+
+    protected abstract List<ISearchPredicate> createSearchPredicates() throws Exception;
+
+    protected abstract IIndexAccessor createAccessor() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
index 0e9c501..934ab65 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
@@ -16,17 +16,15 @@
  ! specific language governing permissions and limitations
  ! under the License.
  !-->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <artifactId>hyracks-storage-am-lsm-btree</artifactId>
-
   <parent>
     <groupId>org.apache.hyracks</groupId>
     <artifactId>hyracks</artifactId>
     <version>0.3.4-SNAPSHOT</version>
   </parent>
-
   <licenses>
     <license>
       <name>Apache License, Version 2.0</name>
@@ -35,11 +33,9 @@
       <comments>A business-friendly OSS license</comments>
     </license>
   </licenses>
-
   <properties>
     <root.dir>${basedir}/../..</root.dir>
   </properties>
-
   <dependencies>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
@@ -92,4 +88,4 @@
       <version>${project.version}</version>
     </dependency>
   </dependencies>
-</project>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
index b0c31ad..0b49b03 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeOpContext.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -73,4 +74,9 @@ public class ExternalBTreeOpContext extends AbstractLSMIndexOperationContext {
     public LSMBTreeCursorInitialState getSearchInitialState() {
         return searchInitialState;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        // No Op
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
index 6993013..0be5556 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -331,9 +331,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
                 numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
                         .getBloomFilter().getNumElements();
             }
-
             componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
-
             try {
                 while (buddyBtreeCursor.hasNext()) {
                     buddyBtreeCursor.next();
@@ -341,7 +339,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
                     componentBulkLoader.delete(tuple);
                 }
             } finally {
-                buddyBtreeCursor.destroy();
+                buddyBtreeCursor.close();
             }
         } else {
             componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
@@ -354,7 +352,7 @@ public class ExternalBTreeWithBuddy extends AbstractLSMIndex implements ITreeInd
                 componentBulkLoader.add(frameTuple);
             }
         } finally {
-            cursor.destroy();
+            cursor.close();
         }
         componentBulkLoader.end();
         return mergedComponent;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
index 85d4ab2..1c74275 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddyOpContext.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.lsm.btree.impls;
 
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -75,4 +76,9 @@ public class ExternalBTreeWithBuddyOpContext extends AbstractLSMIndexOperationCo
     public LSMBTreeWithBuddyCursorInitialState getSearchInitialState() {
         return searchInitialState;
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        // No Op
+    }
 }


[2/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
index 68a3984..ff28470 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeSearchCursorTest.java
@@ -60,48 +60,34 @@ import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.MultiComparator;
 import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 
 public class BTreeSearchCursorTest extends AbstractBTreeTest {
-    protected final int fieldCount = 2;
-    protected final ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
-    protected final BTreeTypeAwareTupleWriterFactory tupleWriterFactory =
-            new BTreeTypeAwareTupleWriterFactory(typeTraits, false);
-    protected final ITreeIndexMetadataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-    protected final Random rnd = new Random(50);
-
-    @Override
-    @Before
-    public void setUp() throws HyracksDataException {
-        super.setUp();
-        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
-        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
-    }
+    public static final int FIELD_COUNT = 2;
+    public static final ITypeTraits[] TYPE_TRAITS = { IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS };
+    public static final BTreeTypeAwareTupleWriterFactory TUPLE_WRITER_FACTORY =
+            new BTreeTypeAwareTupleWriterFactory(TYPE_TRAITS, false);
+    public static final ITreeIndexMetadataFrameFactory META_FRAME_FACTORY = new LIFOMetaDataFrameFactory();
+    public static final int KEY_FIELDS_COUNT = 1;
+    public static final IBinaryComparatorFactory[] CMP_FACTORIES =
+            { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
+    public static final ITreeIndexFrameFactory LEAF_FRAME_FACTORY = new BTreeNSMLeafFrameFactory(TUPLE_WRITER_FACTORY);
+    public static final ITreeIndexFrameFactory INTERIOR_FRAME_FACTORY =
+            new BTreeNSMInteriorFrameFactory(TUPLE_WRITER_FACTORY);
+    public static final Random RANDOM = new Random(50);
 
     @Test
     public void uniqueIndexTest() throws Exception {
         if (LOGGER.isInfoEnabled()) {
             LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
         }
-
         IBufferCache bufferCache = harness.getBufferCache();
-
         // declare keys
-        int keyFieldCount = 1;
-        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
-
-        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
-        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
-
-        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, metaFrameFactory);
-
-        BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
-                fieldCount, harness.getFileReference());
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) LEAF_FRAME_FACTORY.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) INTERIOR_FRAME_FACTORY.createFrame();
+        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, META_FRAME_FACTORY);
+        BTree btree = new BTree(bufferCache, freePageManager, INTERIOR_FRAME_FACTORY, LEAF_FRAME_FACTORY, CMP_FACTORIES,
+                FIELD_COUNT, harness.getFileReference());
         btree.create();
         btree.activate();
 
@@ -111,7 +97,7 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
         TreeSet<Integer> uniqueKeys = new TreeSet<>();
         ArrayList<Integer> keys = new ArrayList<>();
         while (uniqueKeys.size() < numKeys) {
-            int key = rnd.nextInt() % maxKey;
+            int key = RANDOM.nextInt() % maxKey;
             uniqueKeys.add(key);
         }
         for (Integer i : uniqueKeys) {
@@ -151,16 +137,16 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(TUPLE_WRITER_FACTORY);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(TUPLE_WRITER_FACTORY);
 
         IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
         IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
 
-        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, metaFrameFactory);
+        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, META_FRAME_FACTORY);
 
         BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
-                fieldCount, harness.getFileReference());
+                FIELD_COUNT, harness.getFileReference());
         btree.create();
         btree.activate();
 
@@ -169,7 +155,7 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
         int maxKey = 10;
         ArrayList<Integer> keys = new ArrayList<>();
         for (int i = 0; i < numKeys; i++) {
-            int k = rnd.nextInt() % maxKey;
+            int k = RANDOM.nextInt() % maxKey;
             keys.add(k);
         }
         Collections.sort(keys);
@@ -207,16 +193,16 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(TUPLE_WRITER_FACTORY);
+        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(TUPLE_WRITER_FACTORY);
 
         IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
         IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
 
-        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, metaFrameFactory);
+        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, META_FRAME_FACTORY);
 
         BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmpFactories,
-                fieldCount, harness.getFileReference());
+                FIELD_COUNT, harness.getFileReference());
         btree.create();
         btree.activate();
         // generate keys
@@ -224,7 +210,7 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
         int maxKey = 10;
         ArrayList<Integer> keys = new ArrayList<>();
         for (int i = 0; i < numKeys; i++) {
-            int k = rnd.nextInt() % maxKey;
+            int k = RANDOM.nextInt() % maxKey;
             keys.add(k);
         }
         Collections.sort(keys);
@@ -248,7 +234,7 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
         btree.destroy();
     }
 
-    public RangePredicate createRangePredicate(int lk, int hk, boolean lowKeyInclusive, boolean highKeyInclusive)
+    public static RangePredicate createRangePredicate(int lk, int hk, boolean lowKeyInclusive, boolean highKeyInclusive)
             throws HyracksDataException {
         // create tuplereferences for search keys
         ITupleReference lowKey = TupleUtils.createIntegerTuple(false, lk);
@@ -290,43 +276,38 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
     public boolean performSearches(ArrayList<Integer> keys, BTree btree, IBTreeLeafFrame leafFrame,
             IBTreeInteriorFrame interiorFrame, int minKey, int maxKey, boolean lowKeyInclusive,
             boolean highKeyInclusive, boolean printExpectedResults) throws Exception {
-
         ArrayList<Integer> results = new ArrayList<>();
         ArrayList<Integer> expectedResults = new ArrayList<>();
-
         for (int i = minKey; i < maxKey; i++) {
             for (int j = minKey; j < maxKey; j++) {
                 results.clear();
                 expectedResults.clear();
-
                 int lowKey = i;
                 int highKey = j;
-
                 RangePredicate rangePred = createRangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive);
                 IndexAccessParameters actx =
                         new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
                 ITreeIndexAccessor indexAccessor = btree.createAccessor(actx);
                 IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
-                indexAccessor.search(rangeCursor, rangePred);
-
                 try {
-                    while (rangeCursor.hasNext()) {
-                        rangeCursor.next();
-                        ITupleReference frameTuple = rangeCursor.getTuple();
-                        ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
-                                frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
-                        DataInput dataIn = new DataInputStream(inStream);
-                        Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
-                        results.add(res);
+                    indexAccessor.search(rangeCursor, rangePred);
+                    try {
+                        while (rangeCursor.hasNext()) {
+                            rangeCursor.next();
+                            ITupleReference frameTuple = rangeCursor.getTuple();
+                            ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
+                                    frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
+                            DataInput dataIn = new DataInputStream(inStream);
+                            Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
+                            results.add(res);
+                        }
+                    } finally {
+                        rangeCursor.close();
                     }
-                } catch (Exception e) {
-                    e.printStackTrace();
                 } finally {
                     rangeCursor.destroy();
                 }
-
                 getExpectedResults(expectedResults, keys, lowKey, highKey, lowKeyInclusive, highKeyInclusive);
-
                 if (printExpectedResults) {
                     if (expectedResults.size() > 0) {
                         char l, u;
@@ -381,10 +362,14 @@ public class BTreeSearchCursorTest extends AbstractBTreeTest {
     }
 
     protected void insertBTree(List<Integer> keys, BTree btree) throws HyracksDataException {
+        staticInsertBTree(keys, btree);
+    }
+
+    public static void staticInsertBTree(List<Integer> keys, BTree btree) throws HyracksDataException {
         IndexAccessParameters actx =
                 new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
         BTreeAccessor accessor = btree.createAccessor(actx);
-        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(FIELD_COUNT);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
         // insert keys into btree

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
index f32bda3..9a1b6bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
@@ -101,75 +101,81 @@ public class BTreeUpdateSearchTest extends AbstractBTreeTest {
         IndexAccessParameters actx =
                 new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
         ITreeIndexAccessor indexAccessor = btree.createAccessor(actx);
+        try {
+            int numInserts = 10000;
+            for (int i = 0; i < numInserts; i++) {
+                int f0 = rnd.nextInt() % 10000;
+                int f1 = 5;
+                TupleUtils.createIntegerTuple(tb, insertTuple, f0, f1);
+                if (LOGGER.isInfoEnabled()) {
+                    if (i % 10000 == 0) {
+                        long end = System.currentTimeMillis();
+                        LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+                    }
+                }
 
-        int numInserts = 10000;
-        for (int i = 0; i < numInserts; i++) {
-            int f0 = rnd.nextInt() % 10000;
-            int f1 = 5;
-            TupleUtils.createIntegerTuple(tb, insertTuple, f0, f1);
-            if (LOGGER.isInfoEnabled()) {
-                if (i % 10000 == 0) {
-                    long end = System.currentTimeMillis();
-                    LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+                try {
+                    indexAccessor.insert(insertTuple);
+                } catch (HyracksDataException hde) {
+                    if (hde.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+                        hde.printStackTrace();
+                        throw hde;
+                    }
                 }
             }
+            long end = System.currentTimeMillis();
+            long duration = end - start;
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("DURATION: " + duration);
+            }
 
+            // Update scan.
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("UPDATE SCAN:");
+            }
+            // Set the cursor to X latch nodes.
+            RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
+            ITreeIndexCursor updateScanCursor = new BTreeRangeSearchCursor(leafFrame, true);
             try {
-                indexAccessor.insert(insertTuple);
-            } catch (HyracksDataException hde) {
-                if (hde.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
-                    hde.printStackTrace();
-                    throw hde;
+                indexAccessor.search(updateScanCursor, nullPred);
+                try {
+                    while (updateScanCursor.hasNext()) {
+                        updateScanCursor.next();
+                        ITupleReference tuple = updateScanCursor.getTuple();
+                        // Change the value field.
+                        IntegerPointable.setInteger(tuple.getFieldData(1), tuple.getFieldStart(1), 10);
+                    }
+                } finally {
+                    updateScanCursor.close();
                 }
+            } finally {
+                updateScanCursor.destroy();
             }
-        }
-        long end = System.currentTimeMillis();
-        long duration = end - start;
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("DURATION: " + duration);
-        }
-
-        // Update scan.
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("UPDATE SCAN:");
-        }
-        // Set the cursor to X latch nodes.
-        ITreeIndexCursor updateScanCursor = new BTreeRangeSearchCursor(leafFrame, true);
-        RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
-        indexAccessor.search(updateScanCursor, nullPred);
-        try {
-            while (updateScanCursor.hasNext()) {
-                updateScanCursor.next();
-                ITupleReference tuple = updateScanCursor.getTuple();
-                // Change the value field.
-                IntegerPointable.setInteger(tuple.getFieldData(1), tuple.getFieldStart(1), 10);
+            // Ordered scan to verify the values.
+            if (LOGGER.isInfoEnabled()) {
+                LOGGER.info("ORDERED SCAN:");
             }
-        } catch (Exception e) {
-            e.printStackTrace();
-        } finally {
-            updateScanCursor.destroy();
-        }
-
-        // Ordered scan to verify the values.
-        if (LOGGER.isInfoEnabled()) {
-            LOGGER.info("ORDERED SCAN:");
-        }
-        // Set the cursor to X latch nodes.
-        ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, true);
-        indexAccessor.search(scanCursor, nullPred);
-        try {
-            while (scanCursor.hasNext()) {
-                scanCursor.next();
-                ITupleReference tuple = scanCursor.getTuple();
-                String rec = TupleUtils.printTuple(tuple, recDescSers);
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info(rec);
+            // Set the cursor to X latch nodes.
+            ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, true);
+            try {
+                indexAccessor.search(scanCursor, nullPred);
+                try {
+                    while (scanCursor.hasNext()) {
+                        scanCursor.next();
+                        ITupleReference tuple = scanCursor.getTuple();
+                        String rec = TupleUtils.printTuple(tuple, recDescSers);
+                        if (LOGGER.isInfoEnabled()) {
+                            LOGGER.info(rec);
+                        }
+                    }
+                } finally {
+                    scanCursor.close();
                 }
+            } finally {
+                scanCursor.destroy();
             }
-        } catch (Exception e) {
-            e.printStackTrace();
         } finally {
-            scanCursor.destroy();
+            indexAccessor.destroy();
         }
         btree.deactivate();
         btree.destroy();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeDiskScanCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeDiskScanCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeDiskScanCursorTest.java
new file mode 100644
index 0000000..8a27198
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeDiskScanCursorTest.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
+import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class DiskBTreeDiskScanCursorTest extends IIndexCursorTest {
+    private static final BTreeTestHarness harness = new BTreeTestHarness();
+    private static DiskBTree btree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        IMetadataPageManager freePageManager =
+                new LinkedMetaDataPageManager(bufferCache, BTreeSearchCursorTest.META_FRAME_FACTORY);
+        btree = new DiskBTree(bufferCache, freePageManager, BTreeSearchCursorTest.INTERIOR_FRAME_FACTORY,
+                BTreeSearchCursorTest.LEAF_FRAME_FACTORY, BTreeSearchCursorTest.CMP_FACTORIES,
+                BTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference());
+        btree.create();
+        btree.activate();
+
+        TreeSet<Integer> uniqueKeys = new TreeSet<>();
+        ArrayList<Integer> keys = new ArrayList<>();
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        while (uniqueKeys.size() < numKeys) {
+            int key = BTreeSearchCursorTest.RANDOM.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+        DiskBTreeSearchCursorTest.bulkLoadBTree(keys, btree);
+    }
+
+    @AfterClass
+    public static void tearDown() throws HyracksDataException {
+        try {
+            btree.deactivate();
+            btree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws HyracksDataException {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        int minKey = -10;
+        int maxKey = 10;
+        for (int i = minKey; i < maxKey; i++) {
+            for (int j = minKey; j < maxKey; j++) {
+                int lowKey = i;
+                int highKey = j;
+                predicates.add(BTreeSearchCursorTest.createRangePredicate(lowKey, highKey, true, true));
+            }
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        IndexAccessParameters actx =
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
+        return btree.createAccessor(actx);
+    }
+
+    @Override
+    protected void open(IIndexAccessor accessor, IIndexCursor cursor, ISearchPredicate predicate)
+            throws HyracksDataException {
+        ((BTreeAccessor) accessor).diskOrderScan((ITreeIndexCursor) cursor);
+    }
+
+    @Override
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        return ((BTreeAccessor) accessor).createDiskOrderScanCursor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreePointSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreePointSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreePointSearchCursorTest.java
new file mode 100644
index 0000000..c2a69e1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreePointSearchCursorTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
+import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class DiskBTreePointSearchCursorTest extends IIndexCursorTest {
+    private static final BTreeTestHarness harness = new BTreeTestHarness();
+    private static DiskBTree btree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        IMetadataPageManager freePageManager =
+                new LinkedMetaDataPageManager(bufferCache, BTreeSearchCursorTest.META_FRAME_FACTORY);
+        btree = new DiskBTree(bufferCache, freePageManager, BTreeSearchCursorTest.INTERIOR_FRAME_FACTORY,
+                BTreeSearchCursorTest.LEAF_FRAME_FACTORY, BTreeSearchCursorTest.CMP_FACTORIES,
+                BTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference());
+        btree.create();
+        btree.activate();
+
+        TreeSet<Integer> uniqueKeys = new TreeSet<>();
+        ArrayList<Integer> keys = new ArrayList<>();
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        while (uniqueKeys.size() < numKeys) {
+            int key = BTreeSearchCursorTest.RANDOM.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+        DiskBTreeSearchCursorTest.bulkLoadBTree(keys, btree);
+    }
+
+    @AfterClass
+    public static void tearDown() throws HyracksDataException {
+        try {
+            btree.deactivate();
+            btree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws HyracksDataException {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        int minKey = -10;
+        int maxKey = 10;
+        for (int i = minKey; i < maxKey; i++) {
+            int lowKey = i;
+            int highKey = i;
+            predicates.add(BTreeSearchCursorTest.createRangePredicate(lowKey, highKey, true, true));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        IndexAccessParameters actx =
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
+        return btree.createAccessor(actx);
+    }
+
+    @Override
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        return ((BTreeAccessor) accessor).createPointCursor(false);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeRangeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeRangeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeRangeSearchCursorTest.java
new file mode 100644
index 0000000..6c1c1bf
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeRangeSearchCursorTest.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
+import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class DiskBTreeRangeSearchCursorTest extends IIndexCursorTest {
+    private static final BTreeTestHarness harness = new BTreeTestHarness();
+    private static DiskBTree btree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        IMetadataPageManager freePageManager =
+                new LinkedMetaDataPageManager(bufferCache, BTreeSearchCursorTest.META_FRAME_FACTORY);
+        btree = new DiskBTree(bufferCache, freePageManager, BTreeSearchCursorTest.INTERIOR_FRAME_FACTORY,
+                BTreeSearchCursorTest.LEAF_FRAME_FACTORY, BTreeSearchCursorTest.CMP_FACTORIES,
+                BTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference());
+        btree.create();
+        btree.activate();
+
+        TreeSet<Integer> uniqueKeys = new TreeSet<>();
+        ArrayList<Integer> keys = new ArrayList<>();
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        while (uniqueKeys.size() < numKeys) {
+            int key = BTreeSearchCursorTest.RANDOM.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+        DiskBTreeSearchCursorTest.bulkLoadBTree(keys, btree);
+    }
+
+    @AfterClass
+    public static void tearDown() throws HyracksDataException {
+        try {
+            btree.deactivate();
+            btree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws HyracksDataException {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        int minKey = -10;
+        int maxKey = 10;
+        for (int i = minKey; i < maxKey; i++) {
+            for (int j = minKey; j < maxKey; j++) {
+                int lowKey = i;
+                int highKey = j;
+                predicates.add(BTreeSearchCursorTest.createRangePredicate(lowKey, highKey, true, true));
+            }
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        IndexAccessParameters actx =
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
+        return btree.createAccessor(actx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
index c7f0425..c77bea0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskBTreeSearchCursorTest.java
@@ -26,10 +26,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.TreeSet;
 
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -37,15 +34,12 @@ import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDese
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import org.apache.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import org.apache.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import org.apache.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
 import org.apache.hyracks.storage.am.btree.impls.DiskBTree;
 import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.common.TestOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -89,29 +83,18 @@ public class DiskBTreeSearchCursorTest extends BTreeSearchCursorTest {
     private void batchPointLookupTest(int numKeys, int maxKey, int minSearchKey, int maxSearchKey) throws Exception {
 
         IBufferCache bufferCache = harness.getBufferCache();
-
-        // declare keys
-        int keyFieldCount = 1;
-        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
-        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
-
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
-
-        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
-        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
-
-        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, metaFrameFactory);
-
-        DiskBTree btree = new DiskBTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory,
-                cmpFactories, fieldCount, harness.getFileReference());
+        IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) LEAF_FRAME_FACTORY.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) INTERIOR_FRAME_FACTORY.createFrame();
+        IMetadataPageManager freePageManager = new LinkedMetaDataPageManager(bufferCache, META_FRAME_FACTORY);
+        DiskBTree btree = new DiskBTree(bufferCache, freePageManager, INTERIOR_FRAME_FACTORY, LEAF_FRAME_FACTORY,
+                CMP_FACTORIES, FIELD_COUNT, harness.getFileReference());
         btree.create();
         btree.activate();
 
         TreeSet<Integer> uniqueKeys = new TreeSet<>();
         ArrayList<Integer> keys = new ArrayList<>();
         while (uniqueKeys.size() < numKeys) {
-            int key = rnd.nextInt() % maxKey;
+            int key = RANDOM.nextInt() % maxKey;
             uniqueKeys.add(key);
         }
         for (Integer i : uniqueKeys) {
@@ -135,58 +118,60 @@ public class DiskBTreeSearchCursorTest extends BTreeSearchCursorTest {
         BTreeAccessor indexAccessor = btree.createAccessor(
                 new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE));
         IIndexCursor pointCursor = indexAccessor.createPointCursor(false);
-        for (int i = minKey; i < maxKey; i++) {
-            results.clear();
-            expectedResults.clear();
-
-            int lowKey = i;
-            int highKey = i;
-            RangePredicate rangePred = createRangePredicate(lowKey, highKey, true, true);
-            indexAccessor.search(pointCursor, rangePred);
-            try {
-                while (pointCursor.hasNext()) {
-                    pointCursor.next();
-                    ITupleReference frameTuple = pointCursor.getTuple();
-                    ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
-                            frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
-                    DataInput dataIn = new DataInputStream(inStream);
-                    Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
-                    results.add(res);
+        try {
+            for (int i = minKey; i < maxKey; i++) {
+                results.clear();
+                expectedResults.clear();
+                int lowKey = i;
+                int highKey = i;
+                RangePredicate rangePred = createRangePredicate(lowKey, highKey, true, true);
+                indexAccessor.search(pointCursor, rangePred);
+                try {
+                    while (pointCursor.hasNext()) {
+                        pointCursor.next();
+                        ITupleReference frameTuple = pointCursor.getTuple();
+                        ByteArrayInputStream inStream = new ByteArrayInputStream(frameTuple.getFieldData(0),
+                                frameTuple.getFieldStart(0), frameTuple.getFieldLength(0));
+                        DataInput dataIn = new DataInputStream(inStream);
+                        Integer res = IntegerSerializerDeserializer.INSTANCE.deserialize(dataIn);
+                        results.add(res);
+                    }
+                } finally {
+                    pointCursor.close();
                 }
-            } catch (Exception e) {
-                e.printStackTrace();
-            }
-
-            getExpectedResults(expectedResults, keys, lowKey, highKey, true, true);
-
-            if (results.size() == expectedResults.size()) {
-                for (int k = 0; k < results.size(); k++) {
-                    if (!results.get(k).equals(expectedResults.get(k))) {
-                        if (LOGGER.isInfoEnabled()) {
-                            LOGGER.info("DIFFERENT RESULTS AT: i=" + i + " k=" + k);
-                            LOGGER.info(results.get(k) + " " + expectedResults.get(k));
+                getExpectedResults(expectedResults, keys, lowKey, highKey, true, true);
+                if (results.size() == expectedResults.size()) {
+                    for (int k = 0; k < results.size(); k++) {
+                        if (!results.get(k).equals(expectedResults.get(k))) {
+                            if (LOGGER.isInfoEnabled()) {
+                                LOGGER.info("DIFFERENT RESULTS AT: i=" + i + " k=" + k);
+                                LOGGER.info(results.get(k) + " " + expectedResults.get(k));
+                            }
+                            return false;
                         }
-                        return false;
                     }
+                } else {
+                    if (LOGGER.isInfoEnabled()) {
+                        LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i);
+                        LOGGER.info("RESULTS: " + results.size());
+                        LOGGER.info("EXPECTED RESULTS: " + expectedResults.size());
+                    }
+                    return false;
                 }
-            } else {
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i);
-                    LOGGER.info("RESULTS: " + results.size());
-                    LOGGER.info("EXPECTED RESULTS: " + expectedResults.size());
-                }
-                return false;
             }
+        } finally {
+            pointCursor.destroy();;
         }
-
-        pointCursor.close();
-
         return true;
     }
 
     @Override
     protected void insertBTree(List<Integer> keys, BTree btree) throws HyracksDataException {
-        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
+        bulkLoadBTree(keys, btree);
+    }
+
+    public static void bulkLoadBTree(List<Integer> keys, BTree btree) throws HyracksDataException {
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(FIELD_COUNT);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
         IIndexBulkLoader bulkloader = btree.createBulkLoader(1, true, 0, true);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskOrderScanCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskOrderScanCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskOrderScanCursorTest.java
new file mode 100644
index 0000000..7cfd0fe
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/DiskOrderScanCursorTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class DiskOrderScanCursorTest extends IIndexCursorTest {
+    private static final BTreeTestHarness harness = new BTreeTestHarness();
+    private static BTree btree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        IMetadataPageManager freePageManager =
+                new LinkedMetaDataPageManager(bufferCache, BTreeSearchCursorTest.META_FRAME_FACTORY);
+        btree = new BTree(bufferCache, freePageManager, BTreeSearchCursorTest.INTERIOR_FRAME_FACTORY,
+                BTreeSearchCursorTest.LEAF_FRAME_FACTORY, BTreeSearchCursorTest.CMP_FACTORIES,
+                BTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference());
+        btree.create();
+        btree.activate();
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        TreeSet<Integer> uniqueKeys = new TreeSet<>();
+        ArrayList<Integer> keys = new ArrayList<>();
+        while (uniqueKeys.size() < numKeys) {
+            int key = BTreeSearchCursorTest.RANDOM.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+        BTreeSearchCursorTest.staticInsertBTree(keys, btree);
+    }
+
+    @AfterClass
+    public static void tearDown() throws HyracksDataException {
+        try {
+            btree.deactivate();
+            btree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws HyracksDataException {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        int minKey = -10;
+        int maxKey = 10;
+        for (int i = minKey; i < maxKey; i++) {
+            for (int j = minKey; j < maxKey; j++) {
+                int lowKey = i;
+                int highKey = j;
+                predicates.add(BTreeSearchCursorTest.createRangePredicate(lowKey, highKey, true, true));
+            }
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        return btree.createAccessor(
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE));
+    }
+
+    @Override
+    protected void open(IIndexAccessor accessor, IIndexCursor cursor, ISearchPredicate predicate)
+            throws HyracksDataException {
+        ((BTreeAccessor) accessor).diskOrderScan((ITreeIndexCursor) cursor);
+    }
+
+    @Override
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        return ((BTreeAccessor) accessor).createDiskOrderScanCursor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
index 06a00e0..a67450b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
@@ -56,90 +56,100 @@ public class BTreeTestWorker extends AbstractIndexTestWorker {
         ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor();
         MultiComparator cmp = accessor.getOpContext().getCmp();
         RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp);
-
-        switch (op) {
-            case INSERT:
-                try {
-                    accessor.insert(tuple);
-                } catch (HyracksDataException e) {
-                    if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
-                        // Ignore duplicate keys, since we get random tuples.
-                        throw e;
+        try {
+            switch (op) {
+                case INSERT:
+                    try {
+                        accessor.insert(tuple);
+                    } catch (HyracksDataException e) {
+                        if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+                            // Ignore duplicate keys, since we get random tuples.
+                            throw e;
+                        }
                     }
-                }
-                break;
+                    break;
 
-            case DELETE:
-                // Create a tuple reference with only key fields.
-                deleteTb.reset();
-                for (int i = 0; i < numKeyFields; i++) {
-                    deleteTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
-                }
-                deleteTuple.reset(deleteTb.getFieldEndOffsets(), deleteTb.getByteArray());
-                try {
-                    accessor.delete(deleteTuple);
-                } catch (HyracksDataException e) {
-                    if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
-                        // Ignore non-existant keys, since we get random tuples.
-                        throw e;
+                case DELETE:
+                    // Create a tuple reference with only key fields.
+                    deleteTb.reset();
+                    for (int i = 0; i < numKeyFields; i++) {
+                        deleteTb.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+                    }
+                    deleteTuple.reset(deleteTb.getFieldEndOffsets(), deleteTb.getByteArray());
+                    try {
+                        accessor.delete(deleteTuple);
+                    } catch (HyracksDataException e) {
+                        if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+                            // Ignore non-existant keys, since we get random tuples.
+                            throw e;
+                        }
                     }
-                }
-                break;
+                    break;
 
-            case UPDATE:
-                try {
-                    accessor.update(tuple);
-                } catch (HyracksDataException e) {
-                    // Ignore non-existant keys, since we get random tuples.
-                    if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY
-                            && e.getErrorCode() != ErrorCode.INDEX_NOT_UPDATABLE) {
+                case UPDATE:
+                    try {
+                        accessor.update(tuple);
+                    } catch (HyracksDataException e) {
                         // Ignore non-existant keys, since we get random tuples.
-                        // Ignore not updateable exception due to numKeys == numFields.
-                        throw e;
+                        if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY
+                                && e.getErrorCode() != ErrorCode.INDEX_NOT_UPDATABLE) {
+                            // Ignore non-existant keys, since we get random tuples.
+                            // Ignore not updateable exception due to numKeys == numFields.
+                            throw e;
+                        }
                     }
-                }
-                break;
+                    break;
 
-            case UPSERT:
-                accessor.upsert(tuple);
-                // Upsert should not throw. If it does, there's
-                // a bigger problem and the test should fail.
-                break;
+                case UPSERT:
+                    accessor.upsert(tuple);
+                    // Upsert should not throw. If it does, there's
+                    // a bigger problem and the test should fail.
+                    break;
 
-            case POINT_SEARCH:
-                searchCursor.close();
-                rangePred.setLowKey(tuple, true);
-                rangePred.setHighKey(tuple, true);
-                accessor.search(searchCursor, rangePred);
-                consumeCursorTuples(searchCursor);
-                break;
-
-            case SCAN:
-                searchCursor.close();
-                rangePred.setLowKey(null, true);
-                rangePred.setHighKey(null, true);
-                accessor.search(searchCursor, rangePred);
-                consumeCursorTuples(searchCursor);
-                break;
+                case POINT_SEARCH:
+                    searchCursor.close();
+                    rangePred.setLowKey(tuple, true);
+                    rangePred.setHighKey(tuple, true);
+                    accessor.search(searchCursor, rangePred);
+                    try {
+                        consumeCursorTuples(searchCursor);
+                    } finally {
+                        searchCursor.close();
+                    }
+                    break;
 
-            case DISKORDER_SCAN:
-                diskOrderScanCursor.close();
-                accessor.diskOrderScan(diskOrderScanCursor);
-                consumeCursorTuples(diskOrderScanCursor);
-                break;
+                case SCAN:
+                    searchCursor.close();
+                    rangePred.setLowKey(null, true);
+                    rangePred.setHighKey(null, true);
+                    accessor.search(searchCursor, rangePred);
+                    try {
+                        consumeCursorTuples(searchCursor);
+                    } finally {
+                        searchCursor.close();
+                    }
+                    break;
 
-            default:
-                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+                case DISKORDER_SCAN:
+                    accessor.diskOrderScan(diskOrderScanCursor);
+                    try {
+                        consumeCursorTuples(diskOrderScanCursor);
+                    } finally {
+                        diskOrderScanCursor.close();
+                    }
+                    break;
+                default:
+                    throw new HyracksDataException("Op " + op.toString() + " not supported.");
+            }
+        } finally {
+            searchCursor.destroy();
+            diskOrderScanCursor.destroy();
         }
     }
 
     private void consumeCursorTuples(ITreeIndexCursor cursor) throws HyracksDataException {
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-            }
-        } finally {
-            cursor.destroy();
+        while (cursor.hasNext()) {
+            cursor.next();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
index e5cbab2..46a8347 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -77,6 +77,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-common</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-test-support</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index 0c7eed8..aee7e90 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -19,8 +19,6 @@
 
 package org.apache.hyracks.storage.am.lsm.btree;
 
-import java.util.logging.Level;
-
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
@@ -35,6 +33,7 @@ import org.apache.hyracks.storage.am.btree.OrderedIndexExamplesTest;
 import org.apache.hyracks.storage.am.common.TestOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ITreeIndex;
 import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
 import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
 import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
 import org.apache.hyracks.storage.common.IIndexAccessor;
@@ -50,6 +49,14 @@ public class LSMBTreeExamplesTest extends OrderedIndexExamplesTest {
     protected ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
             int[] bloomFilterKeyFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
             int[] btreeFields, int[] filterFields) throws HyracksDataException {
+        return createTreeIndex(harness, typeTraits, cmpFactories, bloomFilterKeyFields, filterTypeTraits,
+                filterCmpFactories, btreeFields, filterFields);
+    }
+
+    public static LSMBTree createTreeIndex(LSMBTreeTestHarness harness, ITypeTraits[] typeTraits,
+            IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, ITypeTraits[] filterTypeTraits,
+            IBinaryComparatorFactory[] filterCmpFactories, int[] btreeFields, int[] filterFields)
+            throws HyracksDataException {
         return LSMBTreeUtil.createLSMTree(harness.getIOManager(), harness.getVirtualBufferCaches(),
                 harness.getFileReference(), harness.getDiskBufferCache(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
index 4c325c0..475ab9c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
@@ -150,7 +150,7 @@ public class LSMBTreeMergeFailTest {
         }
 
         @Override
-        public boolean hasNext() throws HyracksDataException {
+        public boolean doHasNext() throws HyracksDataException {
             throw new UnsupportedOperationException();
         }
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index 77d52bb..3dfb369 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -126,25 +126,30 @@ public class LSMBTreeSearchOperationCallbackTest extends AbstractSearchOperation
                 if (!insertTaskStarted) {
                     condition.await();
                 }
-
                 // begin a search on [50, +inf), blocking on 75
                 TupleUtils.createIntegerTuple(builder, tuple, 50);
                 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
-                consumeIntTupleRange(77, 150, true, 150);
-
-                // consume tuples [152, 300]
-                consumeIntTupleRange(152, 300, false, -1);
-
-                cursor.destroy();
+                try {
+                    expectedTupleToBeLockedValue = 50;
+                    TupleUtils.createIntegerTuple(builder, expectedTupleToBeLocked, expectedTupleToBeLockedValue);
+                    consumeIntTupleRange(50, 75, true, 76);
+
+                    // consume tuples [77, 150], blocking on 151
+                    consumeIntTupleRange(77, 150, true, 150);
+
+                    // consume tuples [152, 300]
+                    consumeIntTupleRange(152, 300, false, -1);
+                } finally {
+                    cursor.close();
+                }
             } finally {
-                lock.unlock();
+                try {
+                    cursor.destroy();
+                } finally {
+                    lock.unlock();
+                }
             }
 
             return true;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index 035ef98..3a48160 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -27,7 +27,6 @@ import java.io.DataInputStream;
 import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.logging.Level;
 
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -369,13 +368,17 @@ public class LSMBTreeUpdateInPlaceScanDiskComponentsTest extends OrderedIndexTes
         ITreeIndexCursor cursor = btreeAccessor.createDiskOrderScanCursor();
         try {
             btreeAccessor.diskOrderScan(cursor);
-            for (UpdatedCheckTuple t : checkTuples) {
-                if (!t.isUpdated() || !hasOnlyKeys) {
-                    checkReturnedTuple((LSMBTreeTupleReference) getNext(cursor), ctx.getFieldSerdes(), t,
-                            ctx.getKeyFieldCount());
+            try {
+                for (UpdatedCheckTuple t : checkTuples) {
+                    if (!t.isUpdated() || !hasOnlyKeys) {
+                        checkReturnedTuple((LSMBTreeTupleReference) getNext(cursor), ctx.getFieldSerdes(), t,
+                                ctx.getKeyFieldCount());
+                    }
                 }
+                Assert.assertFalse(cursor.hasNext());
+            } finally {
+                cursor.close();
             }
-            Assert.assertFalse(cursor.hasNext());
         } finally {
             cursor.destroy();
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java
new file mode 100644
index 0000000..18d89ca
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.cursor;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.am.lsm.btree.LSMBTreeExamplesTest;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeOpContext;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreePointSearchCursor;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class LSMBTreePointSearchCursorTest extends IIndexCursorTest {
+    public static final int FIELD_COUNT = 2;
+    public static final ITypeTraits[] TYPE_TRAITS = { IntegerPointable.TYPE_TRAITS, IntegerPointable.TYPE_TRAITS };
+    @SuppressWarnings("rawtypes")
+    public static final ISerializerDeserializer[] FIELD_SERDES =
+            { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    public static final int KEY_FIELD_COUNT = 1;
+    public static final IBinaryComparatorFactory[] CMP_FACTORIES =
+            { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) };
+    public static final int[] BLOOM_FILTER_KEY_FIELDS = { 0 };
+    public static final Random RND = new Random(50);
+
+    private static final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private static LSMBTree lsmBtree;
+    private static LSMBTreeOpContext opCtx;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        lsmBtree = LSMBTreeExamplesTest.createTreeIndex(harness, TYPE_TRAITS, CMP_FACTORIES, BLOOM_FILTER_KEY_FIELDS,
+                null, null, null, null);
+        lsmBtree.create();
+        lsmBtree.activate();
+        insertData(lsmBtree);
+    }
+
+    @AfterClass
+    public static void teardown() throws HyracksDataException {
+        try {
+            lsmBtree.deactivate();
+            lsmBtree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws Exception {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            // Build low key.
+            ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(KEY_FIELD_COUNT);
+            ArrayTupleReference lowKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(lowKeyTb, lowKey, -100 + (i * 50));
+
+            // Build high key.
+            ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(KEY_FIELD_COUNT);
+            ArrayTupleReference highKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(highKeyTb, highKey, -100 + (i * 50));
+
+            MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(CMP_FACTORIES, lowKey);
+            MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(CMP_FACTORIES, highKey);
+            predicates.add(new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        opCtx = lsmBtree.createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        return new LSMBTreePointSearchCursor(opCtx);
+    }
+
+    @Override
+    protected void open(IIndexAccessor accessor, IIndexCursor cursor, ISearchPredicate predicate)
+            throws HyracksDataException {
+        opCtx.reset();
+        opCtx.setOperation(IndexOperation.SEARCH);
+        lsmBtree.getOperationalComponents(opCtx);
+        opCtx.getSearchInitialState().reset(predicate, opCtx.getComponentHolder());
+        cursor.open(opCtx.getSearchInitialState(), predicate);
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        return lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+    }
+
+    public static void insertData(ITreeIndex lsmBtree) throws HyracksDataException {
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(FIELD_COUNT);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IndexAccessParameters actx =
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
+        IIndexAccessor indexAccessor = lsmBtree.createAccessor(actx);
+        try {
+            int numInserts = 10000;
+            for (int i = 0; i < numInserts; i++) {
+                int f0 = RND.nextInt() % numInserts;
+                int f1 = 5;
+                TupleUtils.createIntegerTuple(tb, tuple, f0, f1);
+                try {
+                    indexAccessor.insert(tuple);
+                } catch (HyracksDataException e) {
+                    if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+                        e.printStackTrace();
+                        throw e;
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                    throw e;
+                }
+            }
+        } finally {
+            indexAccessor.destroy();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeRangeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeRangeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeRangeSearchCursorTest.java
new file mode 100644
index 0000000..e436596
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreeRangeSearchCursorTest.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.cursor;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
+import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.am.lsm.btree.LSMBTreeExamplesTest;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeOpContext;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class LSMBTreeRangeSearchCursorTest extends IIndexCursorTest {
+
+    private static final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+    private static LSMBTree lsmBtree;
+    private static LSMBTreeOpContext opCtx;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        lsmBtree = LSMBTreeExamplesTest.createTreeIndex(harness, LSMBTreePointSearchCursorTest.TYPE_TRAITS,
+                LSMBTreePointSearchCursorTest.CMP_FACTORIES, LSMBTreePointSearchCursorTest.BLOOM_FILTER_KEY_FIELDS,
+                null, null, null, null);
+        lsmBtree.create();
+        lsmBtree.activate();
+        LSMBTreePointSearchCursorTest.insertData(lsmBtree);
+    }
+
+    @AfterClass
+    public static void teardown() throws HyracksDataException {
+        try {
+            lsmBtree.deactivate();
+            lsmBtree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws Exception {
+        // windows of length = 50
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            // Build low key.
+            ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(LSMBTreePointSearchCursorTest.KEY_FIELD_COUNT);
+            ArrayTupleReference lowKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(lowKeyTb, lowKey, -100 + (i * 50));
+            // Build high key.
+            ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(LSMBTreePointSearchCursorTest.KEY_FIELD_COUNT);
+            ArrayTupleReference highKey = new ArrayTupleReference();
+            TupleUtils.createIntegerTuple(highKeyTb, highKey, -100 + (i * 50) + 50);
+            MultiComparator lowKeySearchCmp =
+                    BTreeUtils.getSearchMultiComparator(LSMBTreePointSearchCursorTest.CMP_FACTORIES, lowKey);
+            MultiComparator highKeySearchCmp =
+                    BTreeUtils.getSearchMultiComparator(LSMBTreePointSearchCursorTest.CMP_FACTORIES, highKey);
+            predicates.add(new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        opCtx = lsmBtree.createOpContext(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+        return new LSMBTreeRangeSearchCursor(opCtx);
+    }
+
+    @Override
+    protected void open(IIndexAccessor accessor, IIndexCursor cursor, ISearchPredicate predicate)
+            throws HyracksDataException {
+        opCtx.reset();
+        opCtx.setOperation(IndexOperation.SEARCH);
+        lsmBtree.getOperationalComponents(opCtx);
+        opCtx.getSearchInitialState().reset(predicate, opCtx.getComponentHolder());
+        cursor.open(opCtx.getSearchInitialState(), predicate);
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        return lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+    }
+
+}


[7/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Posted by am...@apache.org.
[ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

- user model changes: no
- storage format changes: no
- interface changes: yes
  - IIndexCursor.close() is now idempotent and can be called on
    a closed cursor.
  - IIndexCursor.destroy() is now idempotent and can be called
    on a destroyed cursor.
  - Add IIndexAccessor.destroy() letting the accessor know it is
    safe to destroy its reusable cursors and operation contexts.
  - Add IIndexOperationContext.destroy() letting the context
    know that the user is done with it and allow it to release
    resources

details:
- Previously, implementations of the IIndexCursor interface
  didn't enforce the interface contract. This change enforces
  the contract for all the implementations.
- With the enforcement of the contract, all the users of the
  cursors are expected to follow and enforce the expected lifecycle.
- Test cases were added.

Change-Id: I98a7a8b931eb24dbe11bf2bdc61b754ca28ebdf9
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2324
Reviewed-by: Michael Blow <mb...@apache.org>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


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

Branch: refs/heads/master
Commit: 4ff6a36d108182e5f36ceac40a27dd89cc29cd23
Parents: 2f392e8
Author: Abdullah Alamoudi <ba...@gmail.com>
Authored: Mon Feb 12 09:24:15 2018 -0800
Committer: abdullah alamoudi <ba...@gmail.com>
Committed: Mon Feb 12 15:27:53 2018 -0800

----------------------------------------------------------------------
 .../apache/asterix/app/nc/RecoveryManager.java  |  46 +--
 .../message/MetadataNodeRequestMessage.java     |   7 +-
 .../asterix/messaging/CCMessageBroker.java      |   2 +-
 .../asterix/test/storage/DiskIsFullTest.java    |  60 ++--
 .../dataflow/FeedRecordDataFlowController.java  |  35 +-
 .../indexing/ExternalFileIndexAccessor.java     |  13 +-
 .../input/stream/SocketServerInputStream.java   |  37 +--
 .../asterix/external/util/DataflowUtils.java    |  37 ++-
 .../apache/asterix/metadata/MetadataNode.java   | 192 ++++++-----
 .../LSMPrimaryUpsertOperatorNodePushable.java   |  26 +-
 .../management/service/logging/LogManager.java  |  67 ++--
 .../hyracks/api/dataflow/IDestroyable.java      |  35 ++
 .../api/exceptions/HyracksDataException.java    |  32 +-
 .../apache/hyracks/api/util/DestroyUtils.java   |  53 +++
 .../apache/hyracks/api/util/ExceptionUtils.java | 108 +++++++
 .../apache/hyracks/control/cc/job/JobRun.java   |   2 +-
 .../control/common/utils/ExceptionUtils.java    |  86 -----
 .../org/apache/hyracks/control/nc/Task.java     |   2 +-
 .../hyracks/control/nc/work/StartTasksWork.java |   2 +-
 .../std/collectors/InputChannelFrameReader.java |   1 -
 .../tests/unit/EnforcedIndexCursorTest.java     |  49 ---
 .../hyracks/tests/unit/IIndexCursorTest.java    | 263 ---------------
 .../BTreeUpdateSearchOperatorNodePushable.java  |   4 +-
 .../hyracks/storage/am/btree/impls/BTree.java   |  10 +
 .../btree/impls/BTreeCountingSearchCursor.java  |  25 +-
 .../storage/am/btree/impls/BTreeOpContext.java  |  14 +
 .../am/btree/impls/BTreeRangeSearchCursor.java  |  37 ++-
 .../storage/am/btree/impls/DiskBTree.java       |   5 +-
 .../btree/impls/DiskBTreePointSearchCursor.java |   6 +-
 .../btree/impls/DiskBTreeRangeSearchCursor.java |   6 +-
 .../storage/am/btree/test/FramewriterTest.java  |   2 +
 .../hyracks/hyracks-storage-am-common/pom.xml   |  30 +-
 .../am/common/api/IIndexOperationContext.java   |  15 +-
 .../am/common/api/ITreeIndexAccessor.java       |   2 +
 .../dataflow/IndexDropOperatorNodePushable.java |   3 +-
 .../IndexSearchOperatorNodePushable.java        |  61 ++--
 ...eIndexDiskOrderScanOperatorNodePushable.java |  79 +++--
 .../impls/TreeIndexDiskOrderScanCursor.java     |  34 +-
 .../am/common/util/ResourceReleaseUtils.java    | 158 +++++++++
 .../common/test/DummyEnforcedIndexCursor.java   |  59 ++++
 .../am/common/test/EnforcedIndexCursorTest.java |  61 ++++
 .../am/common/test/IIndexCursorTest.java        | 294 +++++++++++++++++
 .../hyracks-storage-am-lsm-btree/pom.xml        |  10 +-
 .../lsm/btree/impls/ExternalBTreeOpContext.java |   6 +
 .../lsm/btree/impls/ExternalBTreeWithBuddy.java |   6 +-
 .../impls/ExternalBTreeWithBuddyOpContext.java  |   6 +
 .../storage/am/lsm/btree/impls/LSMBTree.java    | 175 ++++++----
 .../btree/impls/LSMBTreeCursorInitialState.java |  12 +-
 .../impls/LSMBTreeDiskComponentScanCursor.java  |  33 +-
 .../am/lsm/btree/impls/LSMBTreeOpContext.java   |  17 +
 .../btree/impls/LSMBTreePointSearchCursor.java  |  45 ++-
 .../btree/impls/LSMBTreeRangeSearchCursor.java  |  21 +-
 .../lsm/btree/impls/LSMBTreeSearchCursor.java   |  41 +--
 .../impls/LSMBTreeWithBuddyAbstractCursor.java  |  10 +-
 .../impls/LSMBTreeWithBuddySearchCursor.java    |  22 +-
 .../impls/LSMBTreeWithBuddySortedCursor.java    |  14 +-
 .../btree/impls/LSMBuddyBTreeMergeCursor.java   |   2 +-
 .../am/lsm/common/api/ILSMIndexAccessor.java    |   3 +
 .../impls/ComponentReplacementContext.java      |   5 +
 .../lsm/common/impls/LSMIndexSearchCursor.java  |  15 +-
 .../lsm/common/impls/LSMTreeIndexAccessor.java  |  10 +
 .../am/lsm/common/impls/VirtualBufferCache.java |   5 +
 .../invertedindex/api/IInvertedListCursor.java  |   2 +-
 .../invertedindex/impls/LSMInvertedIndex.java   | 174 +++++-----
 .../impls/LSMInvertedIndexAccessor.java         |  10 +
 ...nvertedIndexDeletedKeysBTreeMergeCursor.java |   2 +-
 .../impls/LSMInvertedIndexOpContext.java        |  15 +
 .../LSMInvertedIndexRangeSearchCursor.java      |  11 +-
 .../impls/LSMInvertedIndexSearchCursor.java     |  25 +-
 .../inmemory/InMemoryInvertedIndexAccessor.java |  17 +
 .../InMemoryInvertedIndexOpContext.java         |  13 +
 .../inmemory/InMemoryInvertedListCursor.java    |  23 +-
 .../FixedSizeElementInvertedListCursor.java     |  26 +-
 .../ondisk/OnDiskInvertedIndex.java             |  60 +++-
 .../ondisk/OnDiskInvertedIndexOpContext.java    |  14 +
 .../OnDiskInvertedIndexRangeSearchCursor.java   |  17 +-
 .../ondisk/OnDiskInvertedIndexSearchCursor.java |  16 +-
 .../ondisk/PartitionedOnDiskInvertedIndex.java  |   3 +-
 .../search/PartitionedTOccurrenceSearcher.java  |   8 +-
 .../hyracks-storage-am-lsm-rtree/pom.xml        |  14 +-
 .../lsm/rtree/impls/ExternalRTreeOpContext.java |   6 +
 .../storage/am/lsm/rtree/impls/LSMRTree.java    | 321 ++++++++++++-------
 .../lsm/rtree/impls/LSMRTreeAbstractCursor.java |  10 +-
 .../LSMRTreeDeletedKeysBTreeMergeCursor.java    |   2 +-
 .../am/lsm/rtree/impls/LSMRTreeOpContext.java   |  18 ++
 .../lsm/rtree/impls/LSMRTreeSearchCursor.java   |  22 +-
 .../lsm/rtree/impls/LSMRTreeSortedCursor.java   |  25 +-
 .../impls/LSMRTreeWithAntiMatterTuples.java     | 186 +++++++----
 ...LSMRTreeWithAntiMatterTuplesFlushCursor.java |  40 ++-
 ...SMRTreeWithAntiMatterTuplesSearchCursor.java |  34 +-
 .../am/lsm/rtree/impls/TreeTupleSorter.java     |  37 ++-
 .../hyracks/storage/am/rtree/impls/RTree.java   |  10 +
 .../storage/am/rtree/impls/RTreeOpContext.java  |  12 +
 .../am/rtree/impls/RTreeSearchCursor.java       |  17 +-
 .../storage/common/EnforcedIndexCursor.java     |  98 ++++--
 .../apache/hyracks/storage/common/IIndex.java   |   2 +-
 .../hyracks/storage/common/IIndexAccessor.java  |  24 +-
 .../storage/common/IIndexBulkLoader.java        |   4 +-
 .../hyracks/storage/common/IIndexCursor.java    |  16 +-
 .../storage/common/buffercache/VirtualPage.java |   5 +-
 .../AbstractSearchOperationCallbackTest.java    |  37 ++-
 .../am/btree/OrderedIndexExamplesTest.java      |  70 ++--
 .../storage/am/btree/OrderedIndexTestUtils.java | 160 ++++-----
 .../am/common/AbstractIndexTestWorker.java      |   8 +-
 .../storage/am/common/TreeIndexTestUtils.java   |  75 +++--
 .../am/rtree/AbstractRTreeExamplesTest.java     |  56 ++--
 .../storage/am/rtree/RTreeTestUtils.java        |  62 ++--
 .../hyracks-storage-am-btree-test/pom.xml       |   7 +
 .../am/btree/BTreeCountingCursorTest.java       | 106 ++++++
 .../am/btree/BTreeRangeSearchCursorTest.java    | 100 ++++++
 .../storage/am/btree/BTreeSearchCursorTest.java | 109 +++----
 .../storage/am/btree/BTreeUpdateSearchTest.java | 122 +++----
 .../am/btree/DiskBTreeDiskScanCursorTest.java   | 115 +++++++
 .../btree/DiskBTreePointSearchCursorTest.java   | 106 ++++++
 .../btree/DiskBTreeRangeSearchCursorTest.java   | 101 ++++++
 .../am/btree/DiskBTreeSearchCursorTest.java     | 113 +++----
 .../am/btree/DiskOrderScanCursorTest.java       | 113 +++++++
 .../am/btree/multithread/BTreeTestWorker.java   | 150 +++++----
 .../hyracks-storage-am-lsm-btree-test/pom.xml   |   7 +
 .../am/lsm/btree/LSMBTreeExamplesTest.java      |  11 +-
 .../am/lsm/btree/LSMBTreeMergeFailTest.java     |   2 +-
 .../LSMBTreeSearchOperationCallbackTest.java    |  31 +-
 ...TreeUpdateInPlaceScanDiskComponentsTest.java |  15 +-
 .../cursor/LSMBTreePointSearchCursorTest.java   | 163 ++++++++++
 .../cursor/LSMBTreeRangeSearchCursorTest.java   | 116 +++++++
 .../btree/cursor/LSMBTreeSearchCursorTest.java  | 104 ++++++
 .../btree/impl/TestLsmBtreeSearchCursor.java    |   4 +-
 .../util/LSMInvertedIndexTestUtils.java         | 170 +++++-----
 .../hyracks-storage-am-rtree-test/pom.xml       |   7 +
 .../rtree/RTreeSearchCursorLifecycleTest.java   |  88 +++++
 .../storage/am/rtree/RTreeSearchCursorTest.java | 149 ++++-----
 .../am/rtree/multithread/RTreeTestWorker.java   |  64 ++--
 hyracks-fullstack/pom.xml                       |  50 ++-
 133 files changed, 4282 insertions(+), 2166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index de3c691..273d832 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -680,33 +680,41 @@ public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
             ILSMIndex index =
                     (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(), logRecord.getResourceId());
             ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            if (logRecord.getNewOp() == AbstractIndexModificationOperationCallback.INSERT_BYTE) {
-                indexAccessor.forceDelete(logRecord.getNewValue());
-            } else if (logRecord.getNewOp() == AbstractIndexModificationOperationCallback.DELETE_BYTE) {
-                indexAccessor.forceInsert(logRecord.getOldValue());
-            } else if (logRecord.getNewOp() == AbstractIndexModificationOperationCallback.UPSERT_BYTE) {
-                // undo, upsert the old value if found, otherwise, physical delete
-                if (logRecord.getOldValue() == null) {
-                    try {
-                        indexAccessor.forcePhysicalDelete(logRecord.getNewValue());
-                    } catch (HyracksDataException hde) {
-                        // Since we're undoing according the write-ahead log, the actual upserting tuple
-                        // might not have been written to memory yet.
-                        if (hde.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
-                            throw hde;
-                        }
-                    }
+            try {
+                if (logRecord.getNewOp() == AbstractIndexModificationOperationCallback.INSERT_BYTE) {
+                    indexAccessor.forceDelete(logRecord.getNewValue());
+                } else if (logRecord.getNewOp() == AbstractIndexModificationOperationCallback.DELETE_BYTE) {
+                    indexAccessor.forceInsert(logRecord.getOldValue());
+                } else if (logRecord.getNewOp() == AbstractIndexModificationOperationCallback.UPSERT_BYTE) {
+                    // undo, upsert the old value if found, otherwise, physical delete
+                    undoUpsert(indexAccessor, logRecord);
                 } else {
-                    indexAccessor.forceUpsert(logRecord.getOldValue());
+                    throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
                 }
-            } else {
-                throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+            } finally {
+                indexAccessor.destroy();
             }
         } catch (Exception e) {
             throw new IllegalStateException("Failed to undo", e);
         }
     }
 
+    private static void undoUpsert(ILSMIndexAccessor indexAccessor, ILogRecord logRecord) throws HyracksDataException {
+        if (logRecord.getOldValue() == null) {
+            try {
+                indexAccessor.forcePhysicalDelete(logRecord.getNewValue());
+            } catch (HyracksDataException hde) {
+                // Since we're undoing according the write-ahead log, the actual upserting tuple
+                // might not have been written to memory yet.
+                if (hde.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+                    throw hde;
+                }
+            }
+        } else {
+            indexAccessor.forceUpsert(logRecord.getOldValue());
+        }
+    }
+
     private static void redo(ILogRecord logRecord, IDatasetLifecycleManager datasetLifecycleManager) {
         try {
             int datasetId = logRecord.getDatasetId();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeRequestMessage.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeRequestMessage.java
index 817bbe6..fb89c9c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/MetadataNodeRequestMessage.java
@@ -24,6 +24,7 @@ import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.INcAddressedMessage;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -44,7 +45,7 @@ public class MetadataNodeRequestMessage extends CcIdentifiedMessage
     @Override
     public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
         INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
-        HyracksDataException hde = null;
+        Throwable hde = null;
         try {
             if (export) {
                 appContext.initializeMetadata(false, partitionId);
@@ -63,11 +64,11 @@ public class MetadataNodeRequestMessage extends CcIdentifiedMessage
                 broker.sendMessageToCC(getCcId(), reponse);
             } catch (Exception e) {
                 LOGGER.log(Level.ERROR, "Failed taking over metadata", e);
-                hde = HyracksDataException.suppress(hde, e);
+                hde = ExceptionUtils.suppress(hde, e);
             }
         }
         if (hde != null) {
-            throw hde;
+            throw HyracksDataException.create(hde);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
index 80e0b33..5f42d7a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
@@ -121,7 +121,7 @@ public class CCMessageBroker implements ICCMessageBroker {
             MutablePair<ResponseState, Object> right = pair.getRight();
             switch (right.getKey()) {
                 case FAILURE:
-                    throw HyracksDataException.create((Exception) right.getValue());
+                    throw HyracksDataException.create((Throwable) right.getValue());
                 case SUCCESS:
                     return right.getRight();
                 default:

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
index 8897169..17509a4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
@@ -33,6 +33,9 @@ import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.TransactionOptions;
 import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.file.StorageComponentProvider;
 import org.apache.asterix.metadata.entities.Dataset;
@@ -128,35 +131,46 @@ public class DiskIsFullTest {
                         KEY_INDICATOR_LIST, 0);
                 JobId jobId = nc.newJobId();
                 IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
+                ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
+                        new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
                 // Prepare insert operation
                 LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
                         RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
-                insertOp.open();
-                TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
-                        RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
-                VSizeFrame frame = new VSizeFrame(ctx);
-                FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
-                // Insert records until disk becomes full
-                int tupleCount = 100000;
-                while (tupleCount > 0) {
-                    ITupleReference tuple = tupleGenerator.next();
-                    try {
-                        DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
-                    } catch (Throwable t) {
-                        final Throwable rootCause = ExceptionUtils.getRootCause(t);
-                        rootCause.printStackTrace();
-                        if (rootCause instanceof HyracksDataException) {
-                            HyracksDataException cause = (HyracksDataException) rootCause;
-                            Assert.assertEquals(cause.getErrorCode(), expectedException.getErrorCode());
-                            Assert.assertEquals(cause.getMessage(), expectedException.getMessage());
-                            return;
-                        } else {
-                            break;
+                try {
+                    insertOp.open();
+                    TupleGenerator tupleGenerator =
+                            new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR, RECORD_GEN_FUNCTION,
+                                    UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+                    VSizeFrame frame = new VSizeFrame(ctx);
+                    FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
+                    // Insert records until disk becomes full
+                    int tupleCount = 100000;
+                    while (tupleCount > 0) {
+                        ITupleReference tuple = tupleGenerator.next();
+                        try {
+                            DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
+                        } catch (Throwable t) {
+                            final Throwable rootCause = ExceptionUtils.getRootCause(t);
+                            rootCause.printStackTrace();
+                            if (rootCause instanceof HyracksDataException) {
+                                HyracksDataException cause = (HyracksDataException) rootCause;
+                                Assert.assertEquals(cause.getErrorCode(), expectedException.getErrorCode());
+                                Assert.assertEquals(cause.getMessage(), expectedException.getMessage());
+                                return;
+                            } else {
+                                break;
+                            }
                         }
+                        tupleCount--;
+                    }
+                    Assert.fail("Expected exception (" + expectedException + ") was not thrown");
+                } finally {
+                    try {
+                        insertOp.close();
+                    } finally {
+                        nc.getTransactionManager().abortTransaction(txnCtx.getTxnId());
                     }
-                    tupleCount--;
                 }
-                Assert.fail("Expected exception (" + expectedException + ") was not thrown");
             } finally {
                 nc.deInit();
             }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 3b9391e..25c71df 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -26,12 +26,15 @@ import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.api.IRawRecord;
 import org.apache.asterix.external.api.IRecordDataParser;
 import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.util.DataflowUtils;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.FeedLogManager;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -74,7 +77,7 @@ public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowControl
                 setState(State.STARTED);
             }
         }
-        Exception failure = null;
+        Throwable failure = null;
         try {
             tupleForwarder.initialize(ctx, writer);
             while (hasNext()) {
@@ -111,7 +114,7 @@ public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowControl
                 failure = e;
                 tupleForwarder.fail();
             }
-        } catch (Exception e) {
+        } catch (Throwable e) {
             failure = e;
             tupleForwarder.fail();
             LOGGER.log(Level.WARN, "Failure while operating a feed source", e);
@@ -174,30 +177,12 @@ public class FeedRecordDataFlowController<T> extends AbstractFeedDataFlowControl
         }
     }
 
-    private Exception finish(Exception failure) {
-        HyracksDataException hde = null;
-        try {
-            recordReader.close();
-        } catch (Exception th) {
-            LOGGER.log(Level.WARN, "Failure during while operating a feed source", th);
-            hde = HyracksDataException.suppress(hde, th);
-        }
-        try {
-            tupleForwarder.close();
-        } catch (Exception th) {
-            hde = HyracksDataException.suppress(hde, th);
-        } finally {
-            closeSignal();
-        }
+    private Throwable finish(Throwable failure) {
+        Throwable th = ResourceReleaseUtils.close(recordReader, null);
+        th = DataflowUtils.close(tupleForwarder, th);
+        closeSignal();
         setState(State.STOPPED);
-        if (hde != null) {
-            if (failure != null) {
-                failure.addSuppressed(hde);
-            } else {
-                return hde;
-            }
-        }
-        return failure;
+        return ExceptionUtils.suppress(failure, th);
     }
 
     private boolean parseAndForward(IRawRecord<? extends T> record) throws IOException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
index 88b8a14..5a9852e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
@@ -35,6 +35,7 @@ import org.apache.asterix.om.types.BuiltinType;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.DestroyUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -42,6 +43,7 @@ import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
 import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
 import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
 import org.apache.hyracks.storage.am.lsm.btree.impls.ExternalBTree;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.common.IIndexCursor;
@@ -135,12 +137,11 @@ public class ExternalFileIndexAccessor {
     }
 
     public void close() throws HyracksDataException {
-        if (index != null) {
-            try {
-                fileIndexSearchCursor.destroy();
-            } finally {
-                indexDataflowHelper.close();
-            }
+        Throwable failure = ResourceReleaseUtils.close(fileIndexSearchCursor, null);
+        failure = DestroyUtils.destroy(failure, fileIndexSearchCursor, fileIndexAccessor);
+        failure = ResourceReleaseUtils.close(indexDataflowHelper, failure);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
index b487b40..471d23f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
@@ -25,6 +25,7 @@ import java.net.Socket;
 
 import org.apache.asterix.external.api.AsterixInputStream;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.util.ResourceReleaseUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -102,34 +103,14 @@ public class SocketServerInputStream extends AsterixInputStream {
 
     @Override
     public synchronized void close() throws IOException {
-        HyracksDataException hde = null;
-        try {
-            if (connectionStream != null) {
-                connectionStream.close();
-            }
-            connectionStream = null;
-        } catch (IOException e) {
-            hde = HyracksDataException.create(e);
-        }
-        try {
-            if (socket != null) {
-                socket.close();
-            }
-            socket = null;
-        } catch (IOException e) {
-            hde = HyracksDataException.suppress(hde, e);
-        }
-        try {
-            if (server != null) {
-                server.close();
-            }
-        } catch (IOException e) {
-            hde = HyracksDataException.suppress(hde, e);
-        } finally {
-            server = null;
-        }
-        if (hde != null) {
-            throw hde;
+        Throwable failure = ResourceReleaseUtils.close(connectionStream, null);
+        connectionStream = null;
+        failure = ResourceReleaseUtils.close(socket, failure);
+        socket = null;
+        failure = ResourceReleaseUtils.close(server, failure);
+        server = null;
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
index 31a223f..438f1df 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/DataflowUtils.java
@@ -20,8 +20,8 @@ package org.apache.asterix.external.util;
 
 import java.util.Map;
 
-import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.external.api.ITupleForwarder;
 import org.apache.asterix.external.api.ITupleForwarder.TupleForwardPolicy;
 import org.apache.asterix.external.dataflow.CounterTimerTupleForwarder;
@@ -30,11 +30,20 @@ import org.apache.asterix.external.dataflow.FrameFullTupleForwarder;
 import org.apache.asterix.external.dataflow.RateControlledTupleForwarder;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 public class DataflowUtils {
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private DataflowUtils() {
+    }
+
     public static void addTupleToFrame(FrameTupleAppender appender, ArrayTupleBuilder tb, IFrameWriter writer)
             throws HyracksDataException {
         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
@@ -80,4 +89,30 @@ public class DataflowUtils {
             }
         }
     }
+
+    /**
+     * Close the ITupleForwarder and suppress any Throwable thrown by the close call.
+     * This method must NEVER throw any Throwable
+     *
+     * @param indexHelper
+     *            the indexHelper to close
+     * @param root
+     *            the first exception encountered during release of resources
+     * @return the root Throwable if not null or a new Throwable if any was thrown, otherwise, it returns null
+     */
+    public static Throwable close(ITupleForwarder tupleForwarder, Throwable root) {
+        if (tupleForwarder != null) {
+            try {
+                tupleForwarder.close();
+            } catch (Throwable th) { // NOSONAR Will be re-thrown
+                try {
+                    LOGGER.log(Level.WARN, "Failure closing a closeable resource", th);
+                } catch (Throwable ignore) { // NOSONAR Logging exception will be ignored
+                    // NOSONAR ignore
+                }
+                root = ExceptionUtils.suppress(root, th);
+            }
+        }
+        return root;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 681bae7..966c99a 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -1270,64 +1270,68 @@ public class MetadataNode implements IMetadataNode {
 
         StringBuilder sb = new StringBuilder();
         try {
+            RangePredicate rangePred = null;
             IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
             String resourceName = index.getFile().toString();
             IIndex indexInstance = datasetLifecycleManager.get(resourceName);
             datasetLifecycleManager.open(resourceName);
             IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
             IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
-
-            RangePredicate rangePred = null;
-            rangePred = new RangePredicate(null, null, true, true, null, null);
-            indexAccessor.search(rangeCursor, rangePred);
             try {
-                while (rangeCursor.hasNext()) {
-                    rangeCursor.next();
-                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+                rangePred = new RangePredicate(null, null, true, true, null, null);
+                indexAccessor.search(rangeCursor, rangePred);
+                try {
+                    while (rangeCursor.hasNext()) {
+                        rangeCursor.next();
+                        sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
+                                new ISerializerDeserializer[] { SerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ASTRING) }));
+                    }
+                } finally {
+                    rangeCursor.close();
                 }
-            } finally {
-                rangeCursor.destroy();
-            }
-            datasetLifecycleManager.close(resourceName);
-
-            index = MetadataPrimaryIndexes.DATASET_DATASET;
-            indexInstance = datasetLifecycleManager.get(resourceName);
-            datasetLifecycleManager.open(resourceName);
-            indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            rangeCursor = indexAccessor.createSearchCursor(false);
-
-            rangePred = null;
-            rangePred = new RangePredicate(null, null, true, true, null, null);
-            indexAccessor.search(rangeCursor, rangePred);
-            try {
-                while (rangeCursor.hasNext()) {
-                    rangeCursor.next();
-                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+                datasetLifecycleManager.close(resourceName);
+                index = MetadataPrimaryIndexes.DATASET_DATASET;
+                indexInstance = datasetLifecycleManager.get(resourceName);
+                datasetLifecycleManager.open(resourceName);
+                indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+                rangeCursor = indexAccessor.createSearchCursor(false);
+                rangePred = null;
+                rangePred = new RangePredicate(null, null, true, true, null, null);
+                indexAccessor.search(rangeCursor, rangePred);
+                try {
+                    while (rangeCursor.hasNext()) {
+                        rangeCursor.next();
+                        sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
+                                new ISerializerDeserializer[] {
+                                        SerializerDeserializerProvider.INSTANCE
+                                                .getSerializerDeserializer(BuiltinType.ASTRING),
+                                        SerializerDeserializerProvider.INSTANCE
+                                                .getSerializerDeserializer(BuiltinType.ASTRING) }));
+                    }
+                } finally {
+                    rangeCursor.close();
                 }
-            } finally {
-                rangeCursor.destroy();
-            }
-            datasetLifecycleManager.close(resourceName);
-
-            index = MetadataPrimaryIndexes.INDEX_DATASET;
-            indexInstance = datasetLifecycleManager.get(resourceName);
-            datasetLifecycleManager.open(resourceName);
-            indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-            rangeCursor = indexAccessor.createSearchCursor(false);
-
-            rangePred = null;
-            rangePred = new RangePredicate(null, null, true, true, null, null);
-            indexAccessor.search(rangeCursor, rangePred);
-            try {
-                while (rangeCursor.hasNext()) {
-                    rangeCursor.next();
-                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+                datasetLifecycleManager.close(resourceName);
+                index = MetadataPrimaryIndexes.INDEX_DATASET;
+                indexInstance = datasetLifecycleManager.get(resourceName);
+                datasetLifecycleManager.open(resourceName);
+                indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+                rangeCursor = indexAccessor.createSearchCursor(false);
+                rangePred = null;
+                rangePred = new RangePredicate(null, null, true, true, null, null);
+                indexAccessor.search(rangeCursor, rangePred);
+                try {
+                    while (rangeCursor.hasNext()) {
+                        rangeCursor.next();
+                        sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+                                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+                                SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+                                SerializerDeserializerProvider.INSTANCE
+                                        .getSerializerDeserializer(BuiltinType.ASTRING) }));
+                    }
+                } finally {
+                    rangeCursor.close();
                 }
             } finally {
                 rangeCursor.destroy();
@@ -1351,75 +1355,95 @@ public class MetadataNode implements IMetadataNode {
         IIndex indexInstance = datasetLifecycleManager.get(resourceName);
         datasetLifecycleManager.open(resourceName);
         IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-        IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
-
-        IBinaryComparator[] searchCmps = null;
-        MultiComparator searchCmp = null;
-        RangePredicate rangePred = null;
-        if (searchKey != null) {
-            searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
-            for (int i = 0; i < searchKey.getFieldCount(); i++) {
-                searchCmps[i] = comparatorFactories[i].createBinaryComparator();
+        try {
+            IBinaryComparator[] searchCmps = null;
+            MultiComparator searchCmp = null;
+            if (searchKey != null) {
+                searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
+                for (int i = 0; i < searchKey.getFieldCount(); i++) {
+                    searchCmps[i] = comparatorFactories[i].createBinaryComparator();
+                }
+                searchCmp = new MultiComparator(searchCmps);
             }
-            searchCmp = new MultiComparator(searchCmps);
+            RangePredicate rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
+            search(indexAccessor, rangePred, results, valueExtractor, txnId);
+        } finally {
+            indexAccessor.destroy();
         }
-        rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
-        indexAccessor.search(rangeCursor, rangePred);
+        datasetLifecycleManager.close(resourceName);
+    }
 
+    private <ResultType> void search(IIndexAccessor indexAccessor, RangePredicate rangePred, List<ResultType> results,
+            IValueExtractor<ResultType> valueExtractor, TxnId txnId)
+            throws HyracksDataException, RemoteException, AlgebricksException {
+        IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
         try {
-            while (rangeCursor.hasNext()) {
-                rangeCursor.next();
-                ResultType result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
-                if (result != null) {
-                    results.add(result);
+            indexAccessor.search(rangeCursor, rangePred);
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    ResultType result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
+                    if (result != null) {
+                        results.add(result);
+                    }
                 }
+            } finally {
+                rangeCursor.close();
             }
         } finally {
             rangeCursor.destroy();
         }
-        datasetLifecycleManager.close(resourceName);
     }
 
     @Override
     public void initializeDatasetIdFactory(TxnId txnId) throws AlgebricksException, RemoteException {
-        int mostRecentDatasetId = MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID;
+        int mostRecentDatasetId;
         try {
             String resourceName = MetadataPrimaryIndexes.DATASET_DATASET.getFile().getRelativePath();
             IIndex indexInstance = datasetLifecycleManager.get(resourceName);
             datasetLifecycleManager.open(resourceName);
             try {
-                IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
-                IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
-
-                DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
-                IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
-                RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+                mostRecentDatasetId = getMostRecentDatasetIdFromStoredDatasetIndex(indexInstance, txnId);
+            } finally {
+                datasetLifecycleManager.close(resourceName);
+            }
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+        DatasetIdFactory.initialize(mostRecentDatasetId);
+    }
 
+    private int getMostRecentDatasetIdFromStoredDatasetIndex(IIndex indexInstance, TxnId txnId)
+            throws HyracksDataException, RemoteException, AlgebricksException {
+        DatasetTupleTranslator tupleReaderWriter = tupleTranslatorProvider.getDatasetTupleTranslator(false);
+        IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<>(tupleReaderWriter);
+        RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+        int mostRecentDatasetId = MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID;
+        IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+        try {
+            IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
+            try {
                 indexAccessor.search(rangeCursor, rangePred);
-                int datasetId;
-
                 try {
                     while (rangeCursor.hasNext()) {
                         rangeCursor.next();
                         final ITupleReference ref = rangeCursor.getTuple();
                         final Dataset ds = valueExtractor.getValue(txnId, ref);
-                        datasetId = ds.getDatasetId();
+                        int datasetId = ds.getDatasetId();
                         if (mostRecentDatasetId < datasetId) {
                             mostRecentDatasetId = datasetId;
                         }
                     }
                 } finally {
-                    rangeCursor.destroy();
+                    rangeCursor.close();
                 }
             } finally {
-                datasetLifecycleManager.close(resourceName);
+                rangeCursor.destroy();
             }
-
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+        } finally {
+            indexAccessor.destroy();
         }
-
-        DatasetIdFactory.initialize(mostRecentDatasetId);
+        return mostRecentDatasetId;
     }
 
     // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index 21259cc..6bd9d99 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -140,16 +140,19 @@ public class LSMPrimaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDe
                     resetSearchPredicate(index);
                     if (isFiltered || isDelete || hasSecondaries) {
                         lsmAccessor.search(cursor, searchPred);
-                        if (cursor.hasNext()) {
-                            cursor.next();
-                            prevTuple = cursor.getTuple();
+                        try {
+                            if (cursor.hasNext()) {
+                                cursor.next();
+                                prevTuple = cursor.getTuple();
+                                appendFilterToPrevTuple();
+                                appendPrevRecord();
+                                appendPreviousMeta();
+                                appendFilterToOutput();
+                            } else {
+                                appendPreviousTupleAsMissing();
+                            }
+                        } finally {
                             cursor.close(); // end the search
-                            appendFilterToPrevTuple();
-                            appendPrevRecord();
-                            appendPreviousMeta();
-                            appendFilterToOutput();
-                        } else {
-                            appendPreviousTupleAsMissing();
                         }
                     } else {
                         searchCallback.before(key); // lock
@@ -319,7 +322,6 @@ public class LSMPrimaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDe
         if (isFiltered) {
             writeMissingField();
         }
-        cursor.close();
     }
 
     /**
@@ -362,7 +364,9 @@ public class LSMPrimaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDe
     public void close() throws HyracksDataException {
         try {
             try {
-                cursor.destroy();
+                if (cursor != null) {
+                    cursor.destroy();
+                }
             } finally {
                 writer.close();
             }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index e08bebe..4b154bd 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -352,7 +352,7 @@ public class LogManager implements ILogManager, ILifeCycleComponent {
         try {
             if (fileLogDir.exists()) {
                 List<Long> logFileIds = getLogFileIds();
-                if (logFileIds == null) {
+                if (logFileIds.isEmpty()) {
                     fileId = nextLogFileId;
                     createFileIfNotExists(getLogFilePath(fileId));
                     if (LOGGER.isInfoEnabled()) {
@@ -395,9 +395,11 @@ public class LogManager implements ILogManager, ILifeCycleComponent {
     public void deleteOldLogFiles(long checkpointLSN) {
         Long checkpointLSNLogFileID = getLogFileId(checkpointLSN);
         List<Long> logFileIds = getLogFileIds();
-        if (logFileIds != null) {
+        if (!logFileIds.isEmpty()) {
             //sort log files from oldest to newest
             Collections.sort(logFileIds);
+            // remove the last one not to delete the current log file
+            logFileIds.remove(logFileIds.size() - 1);
             /**
              * At this point, any future LogReader should read from LSN >= checkpointLSN
              */
@@ -412,7 +414,6 @@ public class LogManager implements ILogManager, ILifeCycleComponent {
                             || (txnLogFileId2ReaderCount.containsKey(id) && txnLogFileId2ReaderCount.get(id) > 0)) {
                         break;
                     }
-
                     //delete old log file
                     File file = new File(getLogFilePath(id));
                     file.delete();
@@ -447,12 +448,14 @@ public class LogManager implements ILogManager, ILifeCycleComponent {
     private long deleteAllLogFiles() {
         txnLogFileId2ReaderCount.clear();
         List<Long> logFileIds = getLogFileIds();
-        if (logFileIds != null) {
+        if (!logFileIds.isEmpty()) {
             for (Long id : logFileIds) {
                 File file = new File(getLogFilePath(id));
+                LOGGER.info("Deleting log file: " + file.getAbsolutePath());
                 if (!file.delete()) {
                     throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
                 }
+                LOGGER.info("log file: " + file.getAbsolutePath() + " was deleted successfully");
             }
             return logFileIds.get(logFileIds.size() - 1);
         } else {
@@ -464,29 +467,40 @@ public class LogManager implements ILogManager, ILifeCycleComponent {
         File fileLogDir = new File(logDir);
         String[] logFileNames = null;
         List<Long> logFileIds = null;
-        if (fileLogDir.exists()) {
-            logFileNames = fileLogDir.list(new FilenameFilter() {
-                @Override
-                public boolean accept(File dir, String name) {
-                    if (name.startsWith(logFilePrefix)) {
-                        return true;
-                    }
-                    return false;
-                }
-            });
-            if (logFileNames != null && logFileNames.length != 0) {
-                logFileIds = new ArrayList<>();
-                for (String fileName : logFileNames) {
-                    logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
+        if (!fileLogDir.exists()) {
+            LOGGER.log(Level.INFO, "log dir " + logDir + " doesn't exist.  returning empty list");
+            return Collections.emptyList();
+        }
+        if (!fileLogDir.isDirectory()) {
+            throw new IllegalStateException("log dir " + logDir + " exists but it is not a directory");
+        }
+        logFileNames = fileLogDir.list(new FilenameFilter() {
+            @Override
+            public boolean accept(File dir, String name) {
+                if (name.startsWith(logFilePrefix)) {
+                    return true;
                 }
-                Collections.sort(logFileIds, new Comparator<Long>() {
-                    @Override
-                    public int compare(Long arg0, Long arg1) {
-                        return arg0.compareTo(arg1);
-                    }
-                });
+                return false;
             }
-        }
+        });
+        if (logFileNames == null) {
+            throw new IllegalStateException("listing of log dir (" + logDir + ") files returned null. "
+                    + "Either an IO error occurred or the dir was just deleted by another process/thread");
+        }
+        if (logFileNames.length == 0) {
+            LOGGER.log(Level.INFO, "the log dir (" + logDir + ") is empty. returning empty list");
+            return Collections.emptyList();
+        }
+        logFileIds = new ArrayList<>();
+        for (String fileName : logFileNames) {
+            logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
+        }
+        Collections.sort(logFileIds, new Comparator<Long>() {
+            @Override
+            public int compare(Long arg0, Long arg1) {
+                return arg0.compareTo(arg1);
+            }
+        });
         return logFileIds;
     }
 
@@ -551,7 +565,7 @@ public class LogManager implements ILogManager, ILifeCycleComponent {
     @Override
     public long getReadableSmallestLSN() {
         List<Long> logFileIds = getLogFileIds();
-        if (logFileIds != null) {
+        if (!logFileIds.isEmpty()) {
             return logFileIds.get(0) * logFileSize;
         } else {
             throw new IllegalStateException("Couldn't find any log files.");
@@ -672,7 +686,6 @@ class LogFlusher implements Callable<Boolean> {
     public void terminate() {
         // make sure the LogFlusher thread started before terminating it.
         InvokeUtil.doUninterruptibly(started::acquire);
-
         stopping = true;
 
         // we must tell any active flush, if any, to stop

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IDestroyable.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IDestroyable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IDestroyable.java
new file mode 100644
index 0000000..2e50215
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IDestroyable.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.api.dataflow;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface IDestroyable {
+    /**
+     * Destroy the object and releases any system resources associated
+     * with it. If the object is already destroyed then invoking this
+     * method has no effect.
+     * The behavior of other calls after this method is invoked is undefined
+     *
+     * @throws HyracksDataException
+     */
+    void destroy() throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index d0e4655..6560d1b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -30,12 +30,21 @@ public class HyracksDataException extends HyracksException {
 
     private static final long serialVersionUID = 1L;
 
+    /**
+     * Wrap the failure cause in a HyracksDataException.
+     * If the cause is an InterruptedException, the thread is interrupted first.
+     * If the cause is already a HyracksDataException, then return it as it is.
+     *
+     * @param cause
+     *            the root failure
+     * @return the wrapped failure
+     */
     public static HyracksDataException create(Throwable cause) {
-        if (cause instanceof HyracksDataException || cause == null) {
+        if (cause == null) {
+            throw new NullPointerException("Attempt to wrap null in a HyracksDataException");
+        }
+        if (cause instanceof HyracksDataException) {
             return (HyracksDataException) cause;
-        } else if (cause instanceof Error) {
-            // don't wrap errors, allow them to propagate
-            throw (Error) cause;
         } else if (cause instanceof InterruptedException) {
             Thread.currentThread().interrupt();
         }
@@ -50,21 +59,6 @@ public class HyracksDataException extends HyracksException {
         return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
     }
 
-    public static HyracksDataException suppress(HyracksDataException root, Throwable th) {
-        if (root == null) {
-            return HyracksDataException.create(th);
-        }
-        if (th instanceof Error) {
-            // don't suppress errors into a HyracksDataException, allow them to propagate
-            th.addSuppressed(root);
-            throw (Error) th;
-        } else if (th instanceof InterruptedException) {
-            Thread.currentThread().interrupt();
-        }
-        root.addSuppressed(th);
-        return root;
-    }
-
     public HyracksDataException(String component, int errorCode, String message, Throwable cause, String nodeId,
             Serializable... params) {
         super(component, errorCode, message, cause, nodeId, params);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/DestroyUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/DestroyUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/DestroyUtils.java
new file mode 100644
index 0000000..97c284d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/DestroyUtils.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.util;
+
+import org.apache.hyracks.api.dataflow.IDestroyable;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class DestroyUtils {
+
+    private static final Logger LOGGER = LogManager.getLogger();
+
+    private DestroyUtils() {
+    }
+
+    public static Throwable destroy(Throwable root, IDestroyable... destroyables) {
+        for (int i = 0; i < destroyables.length; i++) {
+            if (destroyables[i] != null) {
+                IDestroyable destroyable = destroyables[i];
+                if (destroyable != null) {
+                    try {
+                        destroyable.destroy();
+                    } catch (Throwable th) { // NOSONAR. Had to be done to satisfy contracts
+                        try {
+                            LOGGER.log(Level.WARN, "Failure destroying a destroyable resource", th);
+                        } catch (Throwable ignore) {
+                            // Do nothing
+                        }
+                        root = ExceptionUtils.suppress(root, th);
+                    }
+                }
+            }
+        }
+        return root;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
new file mode 100644
index 0000000..1a88e46
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * @author yingyib
+ */
+public class ExceptionUtils {
+
+    private ExceptionUtils() {
+    }
+
+    /**
+     * get a list of possible root causes from a list of all exceptions
+     *
+     * @param allExceptions
+     * @return List of possible root causes
+     */
+    public static List<Exception> getActualExceptions(List<Exception> allExceptions) {
+        List<Exception> exceptions = new ArrayList<>();
+        for (Exception exception : allExceptions) {
+            if (possibleRootCause(exception)) {
+                exceptions.add(exception);
+            }
+        }
+        return exceptions;
+    }
+
+    /**
+     * Associate a node with a list of exceptions
+     *
+     * @param exceptions
+     * @param nodeId
+     */
+    public static void setNodeIds(Collection<Exception> exceptions, String nodeId) {
+        List<Exception> newExceptions = new ArrayList<>();
+        for (Exception e : exceptions) {
+            if (e instanceof HyracksDataException) {
+                if (((HyracksDataException) e).getNodeId() == null) {
+                    newExceptions.add(HyracksDataException.create((HyracksDataException) e, nodeId));
+                } else {
+                    newExceptions.add(e);
+                }
+            } else {
+                newExceptions.add(new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.FAILURE_ON_NODE, e, nodeId));
+            }
+        }
+        exceptions.clear();
+        exceptions.addAll(newExceptions);
+    }
+
+    private static boolean possibleRootCause(Throwable exception) {
+        Throwable cause = exception;
+        while ((cause = cause.getCause()) != null) {
+            if (cause instanceof java.lang.InterruptedException
+                    || cause instanceof java.nio.channels.ClosedChannelException) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Suppress the second exception if not null into the first exception if not null.
+     * If the suppressed exception is an instance of InterruptedException, the current thread is interrupted.
+     *
+     * @param first
+     *            the root failure
+     * @param second
+     *            the subsequent failure
+     * @return the root exception, or null if both parameters are null
+     */
+    public static Throwable suppress(Throwable first, Throwable second) {
+        if (second != null && second instanceof InterruptedException) {
+            Thread.currentThread().interrupt();
+        }
+        if (first == null) {
+            return second;
+        } else if (second == null) {
+            return first;
+        }
+        first.addSuppressed(second);
+        return first;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
index fa08420..e4699c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/job/JobRun.java
@@ -45,13 +45,13 @@ import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.hyracks.control.cc.DeployedJobSpecStore.DeployedJobSpecDescriptor;
 import org.apache.hyracks.control.cc.executor.ActivityPartitionDetails;
 import org.apache.hyracks.control.cc.executor.JobExecutor;
 import org.apache.hyracks.control.cc.partitions.PartitionMatchMaker;
 import org.apache.hyracks.control.common.job.profiling.om.JobProfile;
-import org.apache.hyracks.control.common.utils.ExceptionUtils;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java
deleted file mode 100644
index 1d506ca..0000000
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/utils/ExceptionUtils.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.control.common.utils;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * @author yingyib
- */
-public class ExceptionUtils {
-
-    private ExceptionUtils() {
-    }
-
-    /**
-     * get a list of possible root causes from a list of all exceptions
-     *
-     * @param allExceptions
-     * @return List of possible root causes
-     */
-    public static List<Exception> getActualExceptions(List<Exception> allExceptions) {
-        List<Exception> exceptions = new ArrayList<>();
-        for (Exception exception : allExceptions) {
-            if (possibleRootCause(exception)) {
-                exceptions.add(exception);
-            }
-        }
-        return exceptions;
-    }
-
-    /**
-     * Associate a node with a list of exceptions
-     *
-     * @param exceptions
-     * @param nodeId
-     */
-    public static void setNodeIds(Collection<Exception> exceptions, String nodeId) {
-        List<Exception> newExceptions = new ArrayList<>();
-        for (Exception e : exceptions) {
-            if (e instanceof HyracksDataException) {
-                if (((HyracksDataException) e).getNodeId() == null) {
-                    newExceptions.add(HyracksDataException.create((HyracksDataException) e, nodeId));
-                } else {
-                    newExceptions.add(e);
-                }
-            } else {
-                newExceptions.add(new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.FAILURE_ON_NODE, e, nodeId));
-            }
-        }
-        exceptions.clear();
-        exceptions.addAll(newExceptions);
-    }
-
-    private static boolean possibleRootCause(Throwable exception) {
-        Throwable cause = exception;
-        while ((cause = cause.getCause()) != null) {
-            if (cause instanceof java.lang.InterruptedException
-                    || cause instanceof java.nio.channels.ClosedChannelException) {
-                return false;
-            }
-        }
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
index 6e5a58e..9b32cc7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
@@ -56,13 +56,13 @@ import org.apache.hyracks.api.job.profiling.counters.ICounter;
 import org.apache.hyracks.api.job.profiling.counters.ICounterContext;
 import org.apache.hyracks.api.partitions.PartitionId;
 import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.api.util.JavaSerializationUtils;
 import org.apache.hyracks.control.common.job.PartitionState;
 import org.apache.hyracks.control.common.job.profiling.StatsCollector;
 import org.apache.hyracks.control.common.job.profiling.counters.Counter;
 import org.apache.hyracks.control.common.job.profiling.om.PartitionProfile;
 import org.apache.hyracks.control.common.job.profiling.om.TaskProfile;
-import org.apache.hyracks.control.common.utils.ExceptionUtils;
 import org.apache.hyracks.control.nc.io.WorkspaceFileFactory;
 import org.apache.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
 import org.apache.hyracks.control.nc.work.NotifyTaskCompleteWork;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
index 46b176e..e229149 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/StartTasksWork.java
@@ -54,10 +54,10 @@ import org.apache.hyracks.api.job.IJobletEventListenerFactory;
 import org.apache.hyracks.api.job.JobFlag;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.api.util.ExceptionUtils;
 import org.apache.hyracks.comm.channels.NetworkInputChannel;
 import org.apache.hyracks.control.common.deployment.DeploymentUtils;
 import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
-import org.apache.hyracks.control.common.utils.ExceptionUtils;
 import org.apache.hyracks.control.common.work.AbstractWork;
 import org.apache.hyracks.control.nc.Joblet;
 import org.apache.hyracks.control.nc.NodeControllerService;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
index 6748a4d..0efed6f 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
@@ -53,7 +53,6 @@ public class InputChannelFrameReader implements IFrameReader, IInputChannelMonit
             try {
                 wait();
             } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
                 throw HyracksDataException.create(e);
             }
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
deleted file mode 100644
index 8fe689e..0000000
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.tests.unit;
-
-import org.apache.hyracks.storage.common.EnforcedIndexCursor;
-import org.apache.hyracks.storage.common.ICursorInitialState;
-import org.apache.hyracks.storage.common.IIndexCursor;
-import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.mockito.Mockito;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class EnforcedIndexCursorTest extends IIndexCursorTest {
-    @Override
-    protected List<ISearchPredicate> createSearchPredicates() {
-        List<ISearchPredicate> predicates = new ArrayList<>();
-        for (int i = 0; i < 10; i++) {
-            predicates.add(Mockito.mock(ISearchPredicate.class));
-        }
-        return predicates;
-    }
-
-    @Override
-    protected ICursorInitialState createCursorInitialState() {
-        return Mockito.mock(ICursorInitialState.class);
-    }
-
-    @Override
-    protected IIndexCursor createCursor() {
-        return new EnforcedIndexCursor();
-    }
-}


[3/7] asterixdb git commit: [ASTERIXDB-2204][STO] Fix implementations and usages of IIndexCursor

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
index 6fe6b60..9ef305c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
@@ -25,6 +25,7 @@ import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexFrame;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -32,7 +33,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
-public class TreeTupleSorter implements ITreeIndexCursor {
+public class TreeTupleSorter extends EnforcedIndexCursor implements ITreeIndexCursor {
     private final static int INITIAL_SIZE = 1000000;
     private int numTuples;
     private int currentTupleIndex;
@@ -62,13 +63,13 @@ public class TreeTupleSorter implements ITreeIndexCursor {
     }
 
     @Override
-    public void close() {
+    public void doClose() {
         numTuples = 0;
         currentTupleIndex = 0;
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (numTuples <= currentTupleIndex) {
             return false;
         }
@@ -87,12 +88,12 @@ public class TreeTupleSorter implements ITreeIndexCursor {
     }
 
     @Override
-    public void next() {
+    public void doNext() {
         currentTupleIndex++;
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple1;
     }
 
@@ -180,34 +181,32 @@ public class TreeTupleSorter implements ITreeIndexCursor {
     private int compare(int[] tPointers, int tp1, int tp2i, int tp2j) throws HyracksDataException {
         int i1 = tPointers[tp1 * 2];
         int j1 = tPointers[tp1 * 2 + 1];
-
         int i2 = tp2i;
         int j2 = tp2j;
-
         ICachedPage node1 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i1), false);
-        leafFrame1.setPage(node1);
-        ICachedPage node2 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i2), false);
-        leafFrame2.setPage(node2);
-
         try {
-            frameTuple1.resetByTupleOffset(leafFrame1.getBuffer().array(), j1);
-            frameTuple2.resetByTupleOffset(leafFrame2.getBuffer().array(), j2);
-
-            return cmp.selectiveFieldCompare(frameTuple1, frameTuple2, comparatorFields);
-
+            leafFrame1.setPage(node1);
+            ICachedPage node2 = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, i2), false);
+            try {
+                leafFrame2.setPage(node2);
+                frameTuple1.resetByTupleOffset(leafFrame1.getBuffer().array(), j1);
+                frameTuple2.resetByTupleOffset(leafFrame2.getBuffer().array(), j2);
+                return cmp.selectiveFieldCompare(frameTuple1, frameTuple2, comparatorFields);
+            } finally {
+                bufferCache.unpin(node2);
+            }
         } finally {
             bufferCache.unpin(node1);
-            bufferCache.unpin(node2);
         }
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // do nothing
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         // do nothing
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
index 1e71b7f..5582075 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTree.java
@@ -769,6 +769,7 @@ public class RTree extends AbstractTreeIndex {
     public class RTreeAccessor implements ITreeIndexAccessor {
         private RTree rtree;
         private RTreeOpContext ctx;
+        private boolean destroyed = false;
 
         public RTreeAccessor(RTree rtree, IModificationOperationCallback modificationCallback,
                 ISearchOperationCallback searchCallback) {
@@ -895,6 +896,15 @@ public class RTree extends AbstractTreeIndex {
                 }
             }
         }
+
+        @Override
+        public void destroy() throws HyracksDataException {
+            if (destroyed) {
+                return;
+            }
+            destroyed = true;
+            ctx.destroy();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeOpContext.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeOpContext.java
index dc2e83b..46e6b22 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeOpContext.java
@@ -62,6 +62,7 @@ public class RTreeOpContext implements IIndexOperationContext, IExtraPageBlockHe
     private IModificationOperationCallback modificationCallback;
 
     private PermutingTupleReference tupleWithNonIndexFields;
+    private boolean destroyed = false;
 
     public RTreeOpContext(IRTreeLeafFrame leafFrame, IRTreeInteriorFrame interiorFrame, IPageManager freePageManager,
             IBinaryComparatorFactory[] cmpFactories, IModificationOperationCallback modificationCallback) {
@@ -207,4 +208,15 @@ public class RTreeOpContext implements IIndexOperationContext, IExtraPageBlockHe
     public void resetNonIndexFieldsTuple(ITupleReference newValue) {
         tupleWithNonIndexFields.reset(newValue);
     }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (destroyed) {
+            return;
+        }
+        destroyed = true;
+        if (cursor != null) {
+            cursor.destroy();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index 3f4e00a..f85c044 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -25,6 +25,7 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
 import org.apache.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
 import org.apache.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
 import org.apache.hyracks.storage.common.ICursorInitialState;
 import org.apache.hyracks.storage.common.ISearchPredicate;
 import org.apache.hyracks.storage.common.MultiComparator;
@@ -32,7 +33,7 @@ import org.apache.hyracks.storage.common.buffercache.IBufferCache;
 import org.apache.hyracks.storage.common.buffercache.ICachedPage;
 import org.apache.hyracks.storage.common.file.BufferedFileHandle;
 
-public class RTreeSearchCursor implements ITreeIndexCursor {
+public class RTreeSearchCursor extends EnforcedIndexCursor implements ITreeIndexCursor {
 
     private int fileId = -1;
     private ICachedPage page = null;
@@ -62,7 +63,7 @@ public class RTreeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void destroy() throws HyracksDataException {
+    public void doDestroy() throws HyracksDataException {
         if (readLatched) {
             page.releaseReadLatch();
             bufferCache.unpin(page);
@@ -75,7 +76,7 @@ public class RTreeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public ITupleReference getTuple() {
+    public ITupleReference doGetTuple() {
         return frameTuple;
     }
 
@@ -163,7 +164,7 @@ public class RTreeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
+    public boolean doHasNext() throws HyracksDataException {
         if (page == null) {
             return false;
         }
@@ -198,12 +199,12 @@ public class RTreeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void next() throws HyracksDataException {
+    public void doNext() throws HyracksDataException {
         tupleIndex = tupleIndexInc;
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+    public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (this.page != null) {
             this.page.releaseReadLatch();
@@ -239,8 +240,8 @@ public class RTreeSearchCursor implements ITreeIndexCursor {
     }
 
     @Override
-    public void close() throws HyracksDataException {
-        destroy();
+    public void doClose() throws HyracksDataException {
+        doDestroy();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
index 19dfbab..929faff 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
@@ -19,83 +19,117 @@
 
 package org.apache.hyracks.storage.common;
 
+import java.util.Arrays;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
-public class EnforcedIndexCursor implements IIndexCursor {
+public abstract class EnforcedIndexCursor implements IIndexCursor {
     enum State {
         CLOSED,
         OPENED,
         DESTROYED
     }
 
+    private static final boolean STORE_TRACES = false;
+    private static final boolean ENFORCE_NEXT_HAS_NEXT = true;
+    private static final boolean ENFORCE_OPEN_CLOSE_DESTROY = true;
+    private static final Logger LOGGER = LogManager.getLogger();
+
     private State state = State.CLOSED;
+    private StackTraceElement[] openCallStack;
+    private StackTraceElement[] destroyCallStack;
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        if (state != State.CLOSED) {
+    public final void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        if (ENFORCE_OPEN_CLOSE_DESTROY && state != State.CLOSED) {
+            if (STORE_TRACES && destroyCallStack != null) {
+                LOGGER.log(Level.WARN, "The cursor was destroyed in " + Arrays.toString(destroyCallStack));
+            }
             throw new IllegalStateException("Cannot open a cursor in the state " + state);
         }
         doOpen(initialState, searchPred);
         state = State.OPENED;
+        if (STORE_TRACES) {
+            openCallStack = new Throwable().getStackTrace();
+        }
     }
 
-    protected void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
-        // Do nothing
-    }
+    protected abstract void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred)
+            throws HyracksDataException;
 
     @Override
-    public boolean hasNext() throws HyracksDataException {
-        if (state != State.OPENED) {
+    public final boolean hasNext() throws HyracksDataException {
+        if (ENFORCE_NEXT_HAS_NEXT && state != State.OPENED) {
             throw new IllegalStateException("Cannot call hasNext() on a cursor in the state " + state);
         }
         return doHasNext();
     }
 
-    protected boolean doHasNext() throws HyracksDataException {
-        return false;
-    }
+    protected abstract boolean doHasNext() throws HyracksDataException;
 
     @Override
-    public void next() throws HyracksDataException {
-        if (state != State.OPENED) {
+    public final void next() throws HyracksDataException {
+        if (ENFORCE_NEXT_HAS_NEXT && state != State.OPENED) {
             throw new IllegalStateException("Cannot call next() on a cursor in the state " + state);
         }
         doNext();
     }
 
-    protected void doNext() throws HyracksDataException {
-        // Do nothing
-    }
+    protected abstract void doNext() throws HyracksDataException;
 
     @Override
-    public void destroy() throws HyracksDataException {
-        if (state != State.CLOSED) {
-            throw new IllegalStateException("Cannot destroy a cursor in the state " + state);
+    public final void destroy() throws HyracksDataException {
+        if (ENFORCE_OPEN_CLOSE_DESTROY) {
+            if (state == State.DESTROYED) {
+                LOGGER.log(Level.WARN,
+                        "multiple cursor.destroy() call in " + Arrays.toString(new Throwable().getStackTrace()));
+                return;
+            } else if (state != State.CLOSED) {
+                if (STORE_TRACES && openCallStack != null) {
+                    LOGGER.log(Level.WARN, "The cursor was opened in " + Arrays.toString(openCallStack));
+                }
+                throw new IllegalStateException("Cannot destroy a cursor in the state " + state);
+            }
         }
-        doDestroy();
         state = State.DESTROYED;
+        try {
+            doDestroy();
+        } finally {
+            if (ENFORCE_OPEN_CLOSE_DESTROY && STORE_TRACES) {
+                destroyCallStack = new Throwable().getStackTrace();
+            }
+        }
     }
 
-    protected void doDestroy() throws HyracksDataException {
-        // Do nothing
-    }
+    protected abstract void doDestroy() throws HyracksDataException;
 
     @Override
-    public void close() throws HyracksDataException {
-        if (state != State.OPENED) {
-            throw new IllegalStateException("Cannot close a cursor in the state " + state);
+    public final void close() throws HyracksDataException {
+        if (ENFORCE_OPEN_CLOSE_DESTROY) {
+            if (state == State.CLOSED) {
+                return;
+            } else if (state == State.DESTROYED) {
+                throw new IllegalStateException("Cannot close a cursor in the state " + state);
+            }
         }
-        doClose();
         state = State.CLOSED;
+        doClose();
     }
 
-    private void doClose() throws HyracksDataException {
-        // Do nothing
-    }
+    protected abstract void doClose() throws HyracksDataException;
 
     @Override
-    public ITupleReference getTuple() {
-        return null;
+    public final ITupleReference getTuple() {
+        if (state == State.OPENED) {
+            return doGetTuple();
+        } else {
+            return null;
+        }
     }
+
+    protected abstract ITupleReference doGetTuple();
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
index 19b4856..4c0012d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndex.java
@@ -134,7 +134,7 @@ public interface IIndex {
     /**
      * @param fillFactor
      * @param verifyInput
-     * @throws IndexException
+     * @throws HyracksDataException
      */
     public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
             boolean checkIfEmptyIndex) throws HyracksDataException;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessor.java
index 27ca8c8..382aea2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexAccessor.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.common;
 
+import org.apache.hyracks.api.dataflow.IDestroyable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -28,7 +29,7 @@ import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
  * can concurrently operate on the same IIndex (i.e., the IIndex must allow
  * concurrent operations).
  */
-public interface IIndexAccessor {
+public interface IIndexAccessor extends IDestroyable {
     /**
      * Inserts the given tuple.
      *
@@ -36,11 +37,10 @@ public interface IIndexAccessor {
      *            Tuple to be inserted.
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
-     * @throws IndexException
      *             If an index-specific constraint is violated, e.g., the key
      *             already exists.
      */
-    public void insert(ITupleReference tuple) throws HyracksDataException;
+    void insert(ITupleReference tuple) throws HyracksDataException;
 
     /**
      * Updates the tuple in the index matching the given tuple with the new
@@ -51,10 +51,9 @@ public interface IIndexAccessor {
      *            tuples contents.
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
-     * @throws IndexException
      *             If there is no matching tuple in the index.
      */
-    public void update(ITupleReference tuple) throws HyracksDataException;
+    void update(ITupleReference tuple) throws HyracksDataException;
 
     /**
      * Deletes the tuple in the index matching the given tuple.
@@ -63,10 +62,9 @@ public interface IIndexAccessor {
      *            Tuple to be deleted.
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
-     * @throws IndexException
      *             If there is no matching tuple in the index.
      */
-    public void delete(ITupleReference tuple) throws HyracksDataException;
+    void delete(ITupleReference tuple) throws HyracksDataException;
 
     /**
      * This operation is only supported by indexes with the notion of a unique key.
@@ -77,29 +75,29 @@ public interface IIndexAccessor {
      *            Tuple to be deleted.
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
-     * @throws IndexException
      *             If there is no matching tuple in the index.
      *
      */
-    public void upsert(ITupleReference tuple) throws HyracksDataException;
+    void upsert(ITupleReference tuple) throws HyracksDataException;
 
     /**
      * Creates a cursor appropriate for passing into search().
      *
      */
-    public IIndexCursor createSearchCursor(boolean exclusive);
+    IIndexCursor createSearchCursor(boolean exclusive);
 
     /**
      * Open the given cursor for an index search using the given predicate as
      * search condition.
      *
-     * @param icursor
+     * Note: if this call returns successfully, then the cursor is open, otherwise it is not.
+     *
+     * @param cursor
      *            Cursor over the index entries satisfying searchPred.
      * @param searchPred
      *            Search condition.
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
-     * @throws IndexException
      */
-    public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException;
+    void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
index 5c4d3c0..fc54903 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
@@ -37,7 +37,6 @@ public interface IIndexBulkLoader {
     /**
      * Finalize the bulk loading operation in the given context.
      *
-     * @throws IndexException
      * @throws HyracksDataException
      *             If the BufferCache throws while un/pinning or un/latching.
      */
@@ -46,6 +45,9 @@ public interface IIndexBulkLoader {
     /**
      * Release all resources held by this bulkloader, with no guarantee of
      * persisted content.
+     *
+     * @throws HyracksDataException
+     *             If the operation was completed through end() invocation before abort is called
      */
     void abort() throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexCursor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexCursor.java
index b561e25..f704921 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexCursor.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.storage.common;
 
+import org.apache.hyracks.api.dataflow.IDestroyable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -44,12 +45,12 @@ import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
  * <li>DESTROYED</li>
  * </ul>
  * When a cursor object is created, it is in the CLOSED state.
- * CLOSED: The only legal calls are open() --> OPENED, or destroy() --> DESTROYED
+ * CLOSED: Legal calls are open() --> OPENED, or destroy() --> DESTROYED, close() --> no effect
  * OPENED: The only legal calls are hasNext(), next(), or close() --> CLOSED.
  * DESTROYED: All calls are illegal.
  * Cursors must enforce the cursor state machine
  */
-public interface IIndexCursor {
+public interface IIndexCursor extends IDestroyable {
     /**
      * Opens the cursor
      * if open succeeds, close must be called.
@@ -76,15 +77,8 @@ public interface IIndexCursor {
     void next() throws HyracksDataException;
 
     /**
-     * Destroys the cursor allowing for release of resources.
-     * The cursor can't be used anymore after this call.
-     *
-     * @throws HyracksDataException
-     */
-    void destroy() throws HyracksDataException;
-
-    /**
-     * Close the cursor when done with it after a successful open
+     * Close the cursor. If the cursor is already closed then invoking this
+     * method has no effect.
      *
      * @throws HyracksDataException
      */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
index be384e0..d7ec4e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/VirtualPage.java
@@ -141,8 +141,9 @@ public class VirtualPage implements ICachedPage {
     @Override
     public String toString() {
         StringBuilder str = new StringBuilder();
-        str.append("{\"class\":\"" + getClass().getSimpleName() + "\", \"readers\":" + getReadLatchCount()
-                + ",\"writers\":" + (isWriteLatched()));
+        str.append("{\"class\":\"").append(getClass().getSimpleName()).append("\", \"readers\":")
+                .append(getReadLatchCount()).append(",\"writers\":").append(isWriteLatched());
+        str.append(",\"next\":").append(next);
         str.append("}");
         return str.toString();
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
index 368430c..6e7717d 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -110,24 +110,27 @@ public abstract class AbstractSearchOperationCallbackTest extends AbstractOperat
         public Boolean call() throws Exception {
             lock.lock();
             try {
-                if (!insertTaskStarted) {
-                    condition.await();
+                try {
+                    while (!insertTaskStarted) {
+                        condition.await();
+                    }
+                    // begin a search on [101, +inf), blocking on 101
+                    TupleUtils.createIntegerTuple(builder, tuple, 101);
+                    predicate.setLowKey(tuple, true);
+                    predicate.setHighKey(null, true);
+                    accessor.search(cursor, predicate);
+                    try {
+                        consumeIntTupleRange(101, 101, true, 101);
+                        // consume tuples [102, 152], blocking on 151
+                        consumeIntTupleRange(102, 151, true, 152);
+                        // consume tuples [153, 300]
+                        consumeIntTupleRange(153, 300, false, -1);
+                    } finally {
+                        cursor.close();
+                    }
+                } finally {
+                    cursor.destroy();
                 }
-
-                // begin a search on [101, +inf), blocking on 101
-                TupleUtils.createIntegerTuple(builder, tuple, 101);
-                predicate.setLowKey(tuple, true);
-                predicate.setHighKey(null, true);
-                accessor.search(cursor, predicate);
-                consumeIntTupleRange(101, 101, true, 101);
-
-                // consume tuples [102, 152], blocking on 151
-                consumeIntTupleRange(102, 151, true, 152);
-
-                // consume tuples [153, 300]
-                consumeIntTupleRange(153, 300, false, -1);
-
-                cursor.destroy();
             } finally {
                 lock.unlock();
             }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index 9ca3b59..80e443d 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -760,16 +760,20 @@ public abstract class OrderedIndexExamplesTest {
             LOGGER.info("Ordered Scan:");
         }
         IIndexCursor scanCursor = indexAccessor.createSearchCursor(false);
-        RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
-        indexAccessor.search(scanCursor, nullPred);
         try {
-            while (scanCursor.hasNext()) {
-                scanCursor.next();
-                ITupleReference frameTuple = scanCursor.getTuple();
-                String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info(rec);
+            RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
+            indexAccessor.search(scanCursor, nullPred);
+            try {
+                while (scanCursor.hasNext()) {
+                    scanCursor.next();
+                    ITupleReference frameTuple = scanCursor.getTuple();
+                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+                    if (LOGGER.isInfoEnabled()) {
+                        LOGGER.info(rec);
+                    }
                 }
+            } finally {
+                scanCursor.close();
             }
         } finally {
             scanCursor.destroy();
@@ -784,15 +788,17 @@ public abstract class OrderedIndexExamplesTest {
             ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
             TreeIndexDiskOrderScanCursor diskOrderCursor =
                     (TreeIndexDiskOrderScanCursor) treeIndexAccessor.createDiskOrderScanCursor();
-            treeIndexAccessor.diskOrderScan(diskOrderCursor);
             try {
-                while (diskOrderCursor.hasNext()) {
-                    diskOrderCursor.next();
-                    ITupleReference frameTuple = diskOrderCursor.getTuple();
-                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-                    if (LOGGER.isInfoEnabled()) {
+                treeIndexAccessor.diskOrderScan(diskOrderCursor);
+                try {
+                    while (diskOrderCursor.hasNext()) {
+                        diskOrderCursor.next();
+                        ITupleReference frameTuple = diskOrderCursor.getTuple();
+                        String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
                         LOGGER.info(rec);
                     }
+                } finally {
+                    diskOrderCursor.close();
                 }
             } finally {
                 diskOrderCursor.destroy();
@@ -821,24 +827,28 @@ public abstract class OrderedIndexExamplesTest {
             LOGGER.info("Range-Search in: [ " + lowKeyString + ", " + highKeyString + "]");
         }
         IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
-        MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, lowKey);
-        MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, highKey);
-        RangePredicate rangePred;
-        if (minFilterTuple != null && maxFilterTuple != null) {
-            rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp,
-                    minFilterTuple, maxFilterTuple);
-        } else {
-            rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp);
-        }
-        indexAccessor.search(rangeCursor, rangePred);
         try {
-            while (rangeCursor.hasNext()) {
-                rangeCursor.next();
-                ITupleReference frameTuple = rangeCursor.getTuple();
-                String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info(rec);
+            MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, lowKey);
+            MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, highKey);
+            RangePredicate rangePred;
+            if (minFilterTuple != null && maxFilterTuple != null) {
+                rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp,
+                        minFilterTuple, maxFilterTuple);
+            } else {
+                rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp);
+            }
+            indexAccessor.search(rangeCursor, rangePred);
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    ITupleReference frameTuple = rangeCursor.getTuple();
+                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+                    if (LOGGER.isInfoEnabled()) {
+                        LOGGER.info(rec);
+                    }
                 }
+            } finally {
+                rangeCursor.close();
             }
         } finally {
             rangeCursor.destroy();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index bf3c8e5..d3dac3e 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -99,37 +99,42 @@ public class OrderedIndexTestUtils extends TreeIndexTestUtils {
         MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), lowKey);
         MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), highKey);
         IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false);
-        RangePredicate rangePred =
-                new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp);
-        ctx.getIndexAccessor().search(searchCursor, rangePred);
-        // Get the subset of elements from the expected set within given key
-        // range.
-        CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, ctx.getFieldSerdes(), lowKeyCmp.getKeyFieldCount());
-        CheckTuple highKeyCheck =
-                createCheckTupleFromTuple(highKey, ctx.getFieldSerdes(), highKeyCmp.getKeyFieldCount());
-        SortedSet<CheckTuple> expectedSubset = null;
-        if (lowKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()
-                || highKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()) {
-            // Searching on a key prefix (low key or high key or both).
-            expectedSubset =
-                    getPrefixExpectedSubset((TreeSet<CheckTuple>) ctx.getCheckTuples(), lowKeyCheck, highKeyCheck);
-        } else {
-            // Searching on all key fields.
-            expectedSubset = ((TreeSet<CheckTuple>) ctx.getCheckTuples()).subSet(lowKeyCheck, lowKeyInclusive,
-                    highKeyCheck, highKeyInclusive);
-        }
-        Iterator<CheckTuple> checkIter = expectedSubset.iterator();
-        int actualCount = 0;
         try {
-            while (searchCursor.hasNext()) {
-                if (!checkIter.hasNext()) {
-                    fail("Range search returned more answers than expected.\nExpected: " + expectedSubset.size());
+            RangePredicate rangePred =
+                    new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp);
+            int actualCount = 0;
+            SortedSet<CheckTuple> expectedSubset = null;
+            ctx.getIndexAccessor().search(searchCursor, rangePred);
+            try {
+                // Get the subset of elements from the expected set within given key
+                // range.
+                CheckTuple lowKeyCheck =
+                        createCheckTupleFromTuple(lowKey, ctx.getFieldSerdes(), lowKeyCmp.getKeyFieldCount());
+                CheckTuple highKeyCheck =
+                        createCheckTupleFromTuple(highKey, ctx.getFieldSerdes(), highKeyCmp.getKeyFieldCount());
+                if (lowKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()
+                        || highKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()) {
+                    // Searching on a key prefix (low key or high key or both).
+                    expectedSubset = getPrefixExpectedSubset((TreeSet<CheckTuple>) ctx.getCheckTuples(), lowKeyCheck,
+                            highKeyCheck);
+                } else {
+                    // Searching on all key fields.
+                    expectedSubset = ((TreeSet<CheckTuple>) ctx.getCheckTuples()).subSet(lowKeyCheck, lowKeyInclusive,
+                            highKeyCheck, highKeyInclusive);
                 }
-                searchCursor.next();
-                CheckTuple expectedTuple = checkIter.next();
-                ITupleReference tuple = searchCursor.getTuple();
-                compareActualAndExpected(tuple, expectedTuple, ctx.getFieldSerdes());
-                actualCount++;
+                Iterator<CheckTuple> checkIter = expectedSubset.iterator();
+                while (searchCursor.hasNext()) {
+                    if (!checkIter.hasNext()) {
+                        fail("Range search returned more answers than expected.\nExpected: " + expectedSubset.size());
+                    }
+                    searchCursor.next();
+                    CheckTuple expectedTuple = checkIter.next();
+                    ITupleReference tuple = searchCursor.getTuple();
+                    compareActualAndExpected(tuple, expectedTuple, ctx.getFieldSerdes());
+                    actualCount++;
+                }
+            } finally {
+                searchCursor.close();
             }
             if (actualCount < expectedSubset.size()) {
                 fail("Range search returned fewer answers than expected.\nExpected: " + expectedSubset.size()
@@ -146,41 +151,43 @@ public class OrderedIndexTestUtils extends TreeIndexTestUtils {
         }
         OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
         IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false);
-
-        ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(ctx.getKeyFieldCount());
-        ArrayTupleReference lowKey = new ArrayTupleReference();
-        ArrayTupleBuilder highKeyBuilder = new ArrayTupleBuilder(ctx.getKeyFieldCount());
-        ArrayTupleReference highKey = new ArrayTupleReference();
-        RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, null, null);
-
-        // Iterate through expected tuples, and perform a point search in the
-        // BTree to verify the tuple can be reached.
-        for (CheckTuple checkTuple : ctx.getCheckTuples()) {
-            createTupleFromCheckTuple(checkTuple, lowKeyBuilder, lowKey, ctx.getFieldSerdes());
-            createTupleFromCheckTuple(checkTuple, highKeyBuilder, highKey, ctx.getFieldSerdes());
-            MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), lowKey);
-            MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), highKey);
-
-            rangePred.setLowKey(lowKey, true);
-            rangePred.setHighKey(highKey, true);
-            rangePred.setLowKeyComparator(lowKeyCmp);
-            rangePred.setHighKeyComparator(highKeyCmp);
-
-            ctx.getIndexAccessor().search(searchCursor, rangePred);
-
-            try {
-                // We expect exactly one answer.
-                if (searchCursor.hasNext()) {
-                    searchCursor.next();
-                    ITupleReference tuple = searchCursor.getTuple();
-                    compareActualAndExpected(tuple, checkTuple, ctx.getFieldSerdes());
-                }
-                if (searchCursor.hasNext()) {
-                    fail("Point search returned more than one answer.");
+        try {
+            ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(ctx.getKeyFieldCount());
+            ArrayTupleReference lowKey = new ArrayTupleReference();
+            ArrayTupleBuilder highKeyBuilder = new ArrayTupleBuilder(ctx.getKeyFieldCount());
+            ArrayTupleReference highKey = new ArrayTupleReference();
+            RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, null, null);
+
+            // Iterate through expected tuples, and perform a point search in the
+            // BTree to verify the tuple can be reached.
+            for (CheckTuple checkTuple : ctx.getCheckTuples()) {
+                createTupleFromCheckTuple(checkTuple, lowKeyBuilder, lowKey, ctx.getFieldSerdes());
+                createTupleFromCheckTuple(checkTuple, highKeyBuilder, highKey, ctx.getFieldSerdes());
+                MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), lowKey);
+                MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), highKey);
+
+                rangePred.setLowKey(lowKey, true);
+                rangePred.setHighKey(highKey, true);
+                rangePred.setLowKeyComparator(lowKeyCmp);
+                rangePred.setHighKeyComparator(highKeyCmp);
+
+                ctx.getIndexAccessor().search(searchCursor, rangePred);
+                try {
+                    // We expect exactly one answer.
+                    if (searchCursor.hasNext()) {
+                        searchCursor.next();
+                        ITupleReference tuple = searchCursor.getTuple();
+                        compareActualAndExpected(tuple, checkTuple, ctx.getFieldSerdes());
+                    }
+                    if (searchCursor.hasNext()) {
+                        fail("Point search returned more than one answer.");
+                    }
+                } finally {
+                    searchCursor.close();
                 }
-            } finally {
-                searchCursor.destroy();
             }
+        } finally {
+            searchCursor.destroy();
         }
     }
 
@@ -479,25 +486,20 @@ public class OrderedIndexTestUtils extends TreeIndexTestUtils {
     public void checkExpectedResults(IIndexCursor cursor, Collection checkTuples, ISerializerDeserializer[] fieldSerdes,
             int keyFieldCount, Iterator<CheckTuple> checkIter) throws Exception {
         int actualCount = 0;
-        try {
-            while (cursor.hasNext()) {
-                if (!checkIter.hasNext()) {
-                    fail("Ordered scan returned more answers than expected.\nExpected: " + checkTuples.size());
-                }
-                cursor.next();
-                CheckTuple expectedTuple = checkIter.next();
-                ITupleReference tuple = cursor.getTuple();
-                compareActualAndExpected(tuple, expectedTuple, fieldSerdes);
-                actualCount++;
-            }
-            if (actualCount < checkTuples.size()) {
-                fail("Ordered scan returned fewer answers than expected.\nExpected: " + checkTuples.size()
-                        + "\nActual  : " + actualCount);
+        while (cursor.hasNext()) {
+            if (!checkIter.hasNext()) {
+                fail("Ordered scan returned more answers than expected.\nExpected: " + checkTuples.size());
             }
-        } finally {
-            cursor.destroy();
+            cursor.next();
+            CheckTuple expectedTuple = checkIter.next();
+            ITupleReference tuple = cursor.getTuple();
+            compareActualAndExpected(tuple, expectedTuple, fieldSerdes);
+            actualCount++;
+        }
+        if (actualCount < checkTuples.size()) {
+            fail("Ordered scan returned fewer answers than expected.\nExpected: " + checkTuples.size() + "\nActual  : "
+                    + actualCount);
         }
-
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/AbstractIndexTestWorker.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/AbstractIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/AbstractIndexTestWorker.java
index 5248a22..b9123a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/AbstractIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/AbstractIndexTestWorker.java
@@ -68,12 +68,8 @@ public abstract class AbstractIndexTestWorker extends Thread implements ITreeInd
     }
 
     protected void consumeCursorTuples(IIndexCursor cursor) throws HyracksDataException {
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-            }
-        } finally {
-            cursor.destroy();
+        while (cursor.hasNext()) {
+            cursor.next();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
index f0b01a2..a703b0b 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -44,7 +44,6 @@ import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
 import org.apache.hyracks.storage.common.IIndexBulkLoader;
 import org.apache.hyracks.storage.common.IIndexCursor;
 import org.apache.hyracks.storage.common.ISearchPredicate;
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -123,10 +122,19 @@ public abstract class TreeIndexTestUtils {
             LOGGER.info("Testing Scan.");
         }
         IIndexCursor scanCursor = ctx.getIndexAccessor().createSearchCursor(false);
-        ISearchPredicate nullPred = createNullSearchPredicate();
-        ctx.getIndexAccessor().search(scanCursor, nullPred);
-        Iterator<CheckTuple> checkIter = ctx.getCheckTuples().iterator();
-        checkExpectedResults(scanCursor, ctx.getCheckTuples(), ctx.getFieldSerdes(), ctx.getKeyFieldCount(), checkIter);
+        try {
+            ISearchPredicate nullPred = createNullSearchPredicate();
+            ctx.getIndexAccessor().search(scanCursor, nullPred);
+            try {
+                Iterator<CheckTuple> checkIter = ctx.getCheckTuples().iterator();
+                checkExpectedResults(scanCursor, ctx.getCheckTuples(), ctx.getFieldSerdes(), ctx.getKeyFieldCount(),
+                        checkIter);
+            } finally {
+                scanCursor.close();
+            }
+        } finally {
+            scanCursor.destroy();
+        }
     }
 
     public void checkDiskOrderScan(IIndexTestContext ctx) throws Exception {
@@ -135,34 +143,29 @@ public abstract class TreeIndexTestUtils {
                 LOGGER.info("Testing Disk-Order Scan.");
             }
             ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) ctx.getIndexAccessor();
-            ITreeIndexCursor diskOrderCursor = treeIndexAccessor.createDiskOrderScanCursor();
-            treeIndexAccessor.diskOrderScan(diskOrderCursor);
-            int actualCount = 0;
             try {
-                while (diskOrderCursor.hasNext()) {
-                    diskOrderCursor.next();
-                    ITupleReference tuple = diskOrderCursor.getTuple();
-                    CheckTuple checkTuple =
-                            createCheckTupleFromTuple(tuple, ctx.getFieldSerdes(), ctx.getKeyFieldCount());
-                    if (!checkDiskOrderScanResult(tuple, checkTuple, ctx)) {
-                        fail("Disk-order scan returned unexpected answer: " + checkTuple.toString());
-                    }
-                    actualCount++;
-                }
-                if (actualCount < ctx.getCheckTuples().size()) {
-                    fail("Disk-order scan returned fewer answers than expected.\nExpected: "
-                            + ctx.getCheckTuples().size() + "\nActual  : " + actualCount);
-                }
-                if (actualCount > ctx.getCheckTuples().size()) {
-                    fail("Disk-order scan returned more answers than expected.\nExpected: "
-                            + ctx.getCheckTuples().size() + "\nActual  : " + actualCount);
-                }
-            } finally {
+                ITreeIndexCursor diskOrderCursor = treeIndexAccessor.createDiskOrderScanCursor();
                 try {
+                    int actualCount = 0;
+                    treeIndexAccessor.diskOrderScan(diskOrderCursor);
+                    try {
+                        actualCount = scan(ctx, diskOrderCursor);
+                    } finally {
+                        diskOrderCursor.close();
+                    }
+                    if (actualCount < ctx.getCheckTuples().size()) {
+                        fail("Disk-order scan returned fewer answers than expected.\nExpected: "
+                                + ctx.getCheckTuples().size() + "\nActual  : " + actualCount);
+                    }
+                    if (actualCount > ctx.getCheckTuples().size()) {
+                        fail("Disk-order scan returned more answers than expected.\nExpected: "
+                                + ctx.getCheckTuples().size() + "\nActual  : " + actualCount);
+                    }
+                } finally {
                     diskOrderCursor.destroy();
-                } catch (Exception ex) {
-                    LOGGER.log(Level.WARN, "Error during scan cursor close", ex);
                 }
+            } finally {
+                treeIndexAccessor.destroy();
             }
         } catch (UnsupportedOperationException e) {
             // Ignore exception because some indexes, e.g. the LSMTrees, don't
@@ -179,7 +182,19 @@ public abstract class TreeIndexTestUtils {
         }
     }
 
-    @SuppressWarnings("unchecked")
+    private int scan(IIndexTestContext ctx, ITreeIndexCursor diskOrderCursor) throws HyracksDataException {
+        int actualCount = 0;
+        while (diskOrderCursor.hasNext()) {
+            diskOrderCursor.next();
+            ITupleReference tuple = diskOrderCursor.getTuple();
+            CheckTuple checkTuple = createCheckTupleFromTuple(tuple, ctx.getFieldSerdes(), ctx.getKeyFieldCount());
+            if (!checkDiskOrderScanResult(tuple, checkTuple, ctx)) {
+                fail("Disk-order scan returned unexpected answer: " + checkTuple.toString());
+            }
+            actualCount++;
+        }
+        return actualCount;
+    }
 
     public Pair<ITupleReference, ITupleReference> insertIntTuples(IIndexTestContext ctx, int numTuples, Random rnd)
             throws Exception {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index a36acf0..6505675 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -846,16 +846,20 @@ public abstract class AbstractRTreeExamplesTest {
             LOGGER.info("Scan:");
         }
         IIndexCursor scanCursor = indexAccessor.createSearchCursor(false);
-        SearchPredicate nullPred = new SearchPredicate(null, null);
-        indexAccessor.search(scanCursor, nullPred);
         try {
-            while (scanCursor.hasNext()) {
-                scanCursor.next();
-                ITupleReference frameTuple = scanCursor.getTuple();
-                String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info(rec);
+            SearchPredicate nullPred = new SearchPredicate(null, null);
+            indexAccessor.search(scanCursor, nullPred);
+            try {
+                while (scanCursor.hasNext()) {
+                    scanCursor.next();
+                    ITupleReference frameTuple = scanCursor.getTuple();
+                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+                    if (LOGGER.isInfoEnabled()) {
+                        LOGGER.info(rec);
+                    }
                 }
+            } finally {
+                scanCursor.close();
             }
         } finally {
             scanCursor.destroy();
@@ -870,15 +874,17 @@ public abstract class AbstractRTreeExamplesTest {
             ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
             TreeIndexDiskOrderScanCursor diskOrderCursor =
                     (TreeIndexDiskOrderScanCursor) treeIndexAccessor.createDiskOrderScanCursor();
-            treeIndexAccessor.diskOrderScan(diskOrderCursor);
             try {
-                while (diskOrderCursor.hasNext()) {
-                    diskOrderCursor.next();
-                    ITupleReference frameTuple = diskOrderCursor.getTuple();
-                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-                    if (LOGGER.isInfoEnabled()) {
+                treeIndexAccessor.diskOrderScan(diskOrderCursor);
+                try {
+                    while (diskOrderCursor.hasNext()) {
+                        diskOrderCursor.next();
+                        ITupleReference frameTuple = diskOrderCursor.getTuple();
+                        String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
                         LOGGER.info(rec);
                     }
+                } finally {
+                    diskOrderCursor.close();
                 }
             } finally {
                 diskOrderCursor.destroy();
@@ -905,25 +911,27 @@ public abstract class AbstractRTreeExamplesTest {
             String kString = TupleUtils.printTuple(key, fieldSerdes);
             LOGGER.info("Range-Search using key: " + kString);
         }
-        IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
         MultiComparator cmp = RTreeUtils.getSearchMultiComparator(cmpFactories, key);
-
         SearchPredicate rangePred;
         if (minFilterTuple != null && maxFilterTuple != null) {
             rangePred = new SearchPredicate(key, cmp, minFilterTuple, maxFilterTuple);
         } else {
             rangePred = new SearchPredicate(key, cmp);
         }
-
-        indexAccessor.search(rangeCursor, rangePred);
+        IIndexCursor rangeCursor = indexAccessor.createSearchCursor(false);
         try {
-            while (rangeCursor.hasNext()) {
-                rangeCursor.next();
-                ITupleReference frameTuple = rangeCursor.getTuple();
-                String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-                if (LOGGER.isInfoEnabled()) {
-                    LOGGER.info(rec);
+            indexAccessor.search(rangeCursor, rangePred);
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    ITupleReference frameTuple = rangeCursor.getTuple();
+                    String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+                    if (LOGGER.isInfoEnabled()) {
+                        LOGGER.info(rec);
+                    }
                 }
+            } finally {
+                rangeCursor.close();
             }
         } finally {
             rangeCursor.destroy();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
index f48da3a..165bf43 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -72,18 +72,25 @@ public class RTreeTestUtils extends TreeIndexTestUtils {
         MultiComparator cmp = RTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), key);
 
         IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false);
-        SearchPredicate searchPred = new SearchPredicate(key, cmp);
-        ctx.getIndexAccessor().search(searchCursor, searchPred);
-
-        // Get the subset of elements from the expected set within given key
-        // range.
-        RTreeCheckTuple keyCheck =
-                (RTreeCheckTuple) createCheckTupleFromTuple(key, ctx.getFieldSerdes(), cmp.getKeyFieldCount());
+        try {
+            SearchPredicate searchPred = new SearchPredicate(key, cmp);
+            ctx.getIndexAccessor().search(searchCursor, searchPred);
+            try {
+                // Get the subset of elements from the expected set within given key
+                // range.
+                RTreeCheckTuple keyCheck =
+                        (RTreeCheckTuple) createCheckTupleFromTuple(key, ctx.getFieldSerdes(), cmp.getKeyFieldCount());
 
-        HashMultiSet<RTreeCheckTuple> expectedResult = null;
+                HashMultiSet<RTreeCheckTuple> expectedResult = null;
 
-        expectedResult = getRangeSearchExpectedResults(ctx.getCheckTuples(), keyCheck);
-        checkExpectedResults(searchCursor, expectedResult, ctx.getFieldSerdes(), ctx.getKeyFieldCount(), null);
+                expectedResult = getRangeSearchExpectedResults(ctx.getCheckTuples(), keyCheck);
+                checkExpectedResults(searchCursor, expectedResult, ctx.getFieldSerdes(), ctx.getKeyFieldCount(), null);
+            } finally {
+                searchCursor.close();
+            }
+        } finally {
+            searchCursor.destroy();
+        }
     }
 
     @SuppressWarnings("unchecked")
@@ -176,27 +183,22 @@ public class RTreeTestUtils extends TreeIndexTestUtils {
     public void checkExpectedResults(IIndexCursor cursor, Collection checkTuples, ISerializerDeserializer[] fieldSerdes,
             int keyFieldCount, Iterator<CheckTuple> checkIter) throws Exception {
         int actualCount = 0;
-        try {
-            while (cursor.hasNext()) {
-                cursor.next();
-                ITupleReference tuple = cursor.getTuple();
-                RTreeCheckTuple checkTuple =
-                        (RTreeCheckTuple) createCheckTupleFromTuple(tuple, fieldSerdes, keyFieldCount);
-                if (!checkTuples.contains(checkTuple)) {
-                    fail("Scan or range search returned unexpected answer: " + checkTuple.toString());
-                }
-                actualCount++;
-            }
-            if (actualCount < checkTuples.size()) {
-                fail("Scan or range search returned fewer answers than expected.\nExpected: " + checkTuples.size()
-                        + "\nActual  : " + actualCount);
-            }
-            if (actualCount > checkTuples.size()) {
-                fail("Scan or range search returned more answers than expected.\nExpected: " + checkTuples.size()
-                        + "\nActual  : " + actualCount);
+        while (cursor.hasNext()) {
+            cursor.next();
+            ITupleReference tuple = cursor.getTuple();
+            RTreeCheckTuple checkTuple = (RTreeCheckTuple) createCheckTupleFromTuple(tuple, fieldSerdes, keyFieldCount);
+            if (!checkTuples.contains(checkTuple)) {
+                fail("Scan or range search returned unexpected answer: " + checkTuple.toString());
             }
-        } finally {
-            cursor.destroy();
+            actualCount++;
+        }
+        if (actualCount < checkTuples.size()) {
+            fail("Scan or range search returned fewer answers than expected.\nExpected: " + checkTuples.size()
+                    + "\nActual  : " + actualCount);
+        }
+        if (actualCount > checkTuples.size()) {
+            fail("Scan or range search returned more answers than expected.\nExpected: " + checkTuples.size()
+                    + "\nActual  : " + actualCount);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index b618834..45d6989 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -67,6 +67,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-am-common</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeCountingCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeCountingCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeCountingCursorTest.java
new file mode 100644
index 0000000..291ab8a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeCountingCursorTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
+import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class BTreeCountingCursorTest extends IIndexCursorTest {
+    private static final BTreeTestHarness harness = new BTreeTestHarness();
+    private static BTree btree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        IMetadataPageManager freePageManager =
+                new LinkedMetaDataPageManager(bufferCache, BTreeSearchCursorTest.META_FRAME_FACTORY);
+        btree = new BTree(bufferCache, freePageManager, BTreeSearchCursorTest.INTERIOR_FRAME_FACTORY,
+                BTreeSearchCursorTest.LEAF_FRAME_FACTORY, BTreeSearchCursorTest.CMP_FACTORIES,
+                BTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference());
+        btree.create();
+        btree.activate();
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        TreeSet<Integer> uniqueKeys = new TreeSet<>();
+        ArrayList<Integer> keys = new ArrayList<>();
+        while (uniqueKeys.size() < numKeys) {
+            int key = BTreeSearchCursorTest.RANDOM.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+        BTreeSearchCursorTest.staticInsertBTree(keys, btree);
+    }
+
+    @AfterClass
+    public static void tearDown() throws HyracksDataException {
+        try {
+            btree.deactivate();
+            btree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws HyracksDataException {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        int minKey = -10;
+        int maxKey = 10;
+        for (int i = minKey; i < maxKey; i++) {
+            for (int j = minKey; j < maxKey; j++) {
+                int lowKey = i;
+                int highKey = j;
+                predicates.add(BTreeSearchCursorTest.createRangePredicate(lowKey, highKey, true, true));
+            }
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        return btree.createAccessor(
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE));
+    }
+
+    @Override
+    protected IIndexCursor createCursor(IIndexAccessor accessor) {
+        return ((BTreeAccessor) accessor).createCountingSearchCursor();
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/4ff6a36d/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeRangeSearchCursorTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeRangeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeRangeSearchCursorTest.java
new file mode 100644
index 0000000..73a9bb6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeRangeSearchCursorTest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.storage.am.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.btree.util.BTreeTestHarness;
+import org.apache.hyracks.storage.am.common.TestOperationCallback;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetaDataPageManager;
+import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
+import org.apache.hyracks.storage.am.common.test.IIndexCursorTest;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class BTreeRangeSearchCursorTest extends IIndexCursorTest {
+    private static final BTreeTestHarness harness = new BTreeTestHarness();
+    private static BTree btree;
+
+    @BeforeClass
+    public static void setup() throws HyracksDataException {
+        harness.setUp();
+        IBufferCache bufferCache = harness.getBufferCache();
+        IMetadataPageManager freePageManager =
+                new LinkedMetaDataPageManager(bufferCache, BTreeSearchCursorTest.META_FRAME_FACTORY);
+        btree = new BTree(bufferCache, freePageManager, BTreeSearchCursorTest.INTERIOR_FRAME_FACTORY,
+                BTreeSearchCursorTest.LEAF_FRAME_FACTORY, BTreeSearchCursorTest.CMP_FACTORIES,
+                BTreeSearchCursorTest.FIELD_COUNT, harness.getFileReference());
+        btree.create();
+        btree.activate();
+        // generate keys
+        int numKeys = 50;
+        int maxKey = 1000;
+        TreeSet<Integer> uniqueKeys = new TreeSet<>();
+        ArrayList<Integer> keys = new ArrayList<>();
+        while (uniqueKeys.size() < numKeys) {
+            int key = BTreeSearchCursorTest.RANDOM.nextInt() % maxKey;
+            uniqueKeys.add(key);
+        }
+        for (Integer i : uniqueKeys) {
+            keys.add(i);
+        }
+        BTreeSearchCursorTest.staticInsertBTree(keys, btree);
+    }
+
+    @AfterClass
+    public static void tearDown() throws HyracksDataException {
+        try {
+            btree.deactivate();
+            btree.destroy();
+        } finally {
+            harness.tearDown();
+        }
+    }
+
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() throws HyracksDataException {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        int minKey = -10;
+        int maxKey = 10;
+        for (int i = minKey; i < maxKey; i++) {
+            for (int j = minKey; j < maxKey; j++) {
+                int lowKey = i;
+                int highKey = j;
+                predicates.add(BTreeSearchCursorTest.createRangePredicate(lowKey, highKey, true, true));
+            }
+        }
+        return predicates;
+    }
+
+    @Override
+    protected IIndexAccessor createAccessor() throws Exception {
+        IndexAccessParameters actx =
+                new IndexAccessParameters(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
+        return btree.createAccessor(actx);
+    }
+}