Merged fullstack_lsm_staging upto r3336

git-svn-id: https://hyracks.googlecode.com/svn/trunk@3339 123451ca-8445-de46-9d55-352943316053
diff --git a/fullstack/hyracks/hyracks-test-support/pom.xml b/fullstack/hyracks/hyracks-test-support/pom.xml
index 1ddd1df..6b6026a 100644
--- a/fullstack/hyracks/hyracks-test-support/pom.xml
+++ b/fullstack/hyracks/hyracks-test-support/pom.xml
@@ -1,8 +1,6 @@
 <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>edu.uci.ics.hyracks</groupId>
   <artifactId>hyracks-test-support</artifactId>
-  <version>0.2.4-SNAPSHOT</version>
   <name>hyracks-test-support</name>
 
   <parent>
@@ -54,7 +52,7 @@
   	</dependency>
   	<dependency>
   		<groupId>edu.uci.ics.hyracks</groupId>
-  		<artifactId>hyracks-storage-am-invertedindex</artifactId>
+  		<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
   		<version>0.2.4-SNAPSHOT</version>
   		<type>jar</type>
   		<scope>compile</scope>
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
new file mode 100644
index 0000000..30843aa
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractModificationOperationCallbackTest.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+
+public abstract class AbstractModificationOperationCallbackTest extends AbstractOperationCallbackTest {
+
+    protected final ArrayTupleBuilder builder;
+    protected final ArrayTupleReference tuple;
+    protected final IModificationOperationCallback cb;
+
+    protected boolean isFoundNull;
+
+    public AbstractModificationOperationCallbackTest() {
+        this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+        this.tuple = new ArrayTupleReference();
+        this.cb = new VeriyfingModificationCallback();
+        this.isFoundNull = true;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        super.tearDown();
+    }
+
+    @Test
+    public void modificationCallbackTest() throws Exception {
+        IIndexAccessor accessor = index.createAccessor(cb, NoOpOperationCallback.INSTANCE);
+
+        isFoundNull = true;
+        for (int i = 0; i < AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT; i++) {
+            TupleUtils.createIntegerTuple(builder, tuple, i);
+            accessor.insert(tuple);
+        }
+
+        isFoundNull = false;
+        for (int i = 0; i < AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT; i++) {
+            TupleUtils.createIntegerTuple(builder, tuple, i);
+            accessor.upsert(tuple);
+        }
+
+        isFoundNull = false;
+        for (int i = 0; i < AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT; i++) {
+            TupleUtils.createIntegerTuple(builder, tuple, i);
+            accessor.delete(tuple);
+        }
+    }
+
+    private class VeriyfingModificationCallback implements IModificationOperationCallback {
+
+        @Override
+        public void before(ITupleReference tuple) {
+            Assert.assertEquals(0, cmp.compare(AbstractModificationOperationCallbackTest.this.tuple, tuple));
+        }
+
+        @Override
+        public void found(ITupleReference before, ITupleReference after) {
+            if (isFoundNull) {
+                Assert.assertEquals(null, before);
+            } else {
+                Assert.assertEquals(0, cmp.compare(AbstractModificationOperationCallbackTest.this.tuple, before));
+            }
+            Assert.assertEquals(0, cmp.compare(AbstractModificationOperationCallbackTest.this.tuple, after));
+        }
+
+    }
+
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractOperationCallbackTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
new file mode 100644
index 0000000..41dfdfe
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractOperationCallbackTest.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public abstract class AbstractOperationCallbackTest {
+    protected static final int NUM_KEY_FIELDS = 1;
+
+    @SuppressWarnings("rawtypes")
+    protected final ISerializerDeserializer[] keySerdes;
+    protected final MultiComparator cmp;
+    protected final int[] bloomFilterKeyFields;
+
+    protected IIndex index;
+
+    protected abstract void createIndexInstance() throws Exception;
+
+    public AbstractOperationCallbackTest() {
+        this.keySerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE };
+        this.cmp = MultiComparator.create(SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length));
+        bloomFilterKeyFields = new int[NUM_KEY_FIELDS];
+        for (int i = 0; i < NUM_KEY_FIELDS; ++i) {
+            bloomFilterKeyFields[i] = i;
+        }
+    }
+
+    public void setup() throws Exception {
+        createIndexInstance();
+        index.create();
+        index.activate();
+    }
+
+    public void tearDown() throws Exception {
+        index.deactivate();
+        index.destroy();
+    }
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
new file mode 100644
index 0000000..ffa7c9a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java
@@ -0,0 +1,230 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+
+public abstract class AbstractSearchOperationCallbackTest extends AbstractOperationCallbackTest {
+    private static final int NUM_TASKS = 2;
+
+    protected final Lock lock;
+    protected final Condition condition;
+
+    protected ExecutorService executor;
+    protected boolean insertTaskStarted;
+
+    public AbstractSearchOperationCallbackTest() {
+        this.lock = new ReentrantLock(true);
+        this.condition = lock.newCondition();
+        this.insertTaskStarted = false;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        executor = Executors.newFixedThreadPool(NUM_TASKS);
+        super.setup();
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        executor.shutdown();
+        super.tearDown();
+    }
+
+    @Test
+    public void searchCallbackTest() throws Exception {
+        Future<Boolean> insertFuture = executor.submit(new InsertionTask());
+        Future<Boolean> searchFuture = executor.submit(new SearchTask());
+        Assert.assertTrue(searchFuture.get());
+        Assert.assertTrue(insertFuture.get());
+    }
+
+    private class SearchTask implements Callable<Boolean> {
+        private final ISearchOperationCallback cb;
+        private final IIndexAccessor accessor;
+        private final IIndexCursor cursor;
+        private final RangePredicate predicate;
+        private final ArrayTupleBuilder builder;
+        private final ArrayTupleReference tuple;
+
+        private boolean blockOnHigh;
+        private int blockingValue;
+        private int expectedAfterBlock;
+
+        public SearchTask() {
+            this.cb = new SynchronizingSearchOperationCallback();
+            this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb);
+            this.cursor = accessor.createSearchCursor();
+            this.predicate = new RangePredicate();
+            this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.tuple = new ArrayTupleReference();
+
+            this.blockOnHigh = false;
+            this.blockingValue = -1;
+            this.expectedAfterBlock = -1;
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            lock.lock();
+            try {
+                if (!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);
+                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.close();
+            } finally {
+                lock.unlock();
+            }
+
+            return true;
+        }
+
+        private void consumeIntTupleRange(int begin, int end, boolean blockOnHigh, int expectedAfterBlock)
+                throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            for (int i = begin; i <= end; i++) {
+                if (blockOnHigh == true && i == end) {
+                    this.blockOnHigh = true;
+                    this.blockingValue = end;
+                    this.expectedAfterBlock = expectedAfterBlock;
+                }
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                if (!cursor.hasNext()) {
+                    Assert.fail("Failed to consume entire tuple range since cursor is exhausted.");
+                }
+                cursor.next();
+
+                if (this.blockOnHigh) {
+                    TupleUtils.createIntegerTuple(builder, tuple, expectedAfterBlock);
+                }
+                Assert.assertEquals(0, cmp.compare(tuple, cursor.getTuple()));
+            }
+        }
+
+        private class SynchronizingSearchOperationCallback implements ISearchOperationCallback {
+
+            @Override
+            public boolean proceed(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.tuple, tuple));
+                return false;
+            }
+
+            @Override
+            public void reconcile(ITupleReference tuple) {
+                Assert.assertEquals(0, cmp.compare(SearchTask.this.tuple, tuple));
+                if (blockOnHigh) {
+                    try {
+                        TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, expectedAfterBlock);
+                    } catch (HyracksDataException e) {
+                        e.printStackTrace();
+                    }
+                    condition.signal();
+                    condition.awaitUninterruptibly();
+                    blockOnHigh = false;
+                }
+            }
+
+            @Override
+            public void cancel(ITupleReference tuple) {
+                try {
+                    TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, blockingValue);
+                    Assert.assertEquals(0, cmp.compare(tuple, SearchTask.this.tuple));
+                    TupleUtils.createIntegerTuple(builder, SearchTask.this.tuple, expectedAfterBlock);
+                } catch (HyracksDataException e) {
+                    e.printStackTrace();
+                }
+            }
+
+        }
+    }
+
+    private class InsertionTask implements Callable<Boolean> {
+        private final IIndexAccessor accessor;
+        private final ArrayTupleBuilder builder;
+        private final ArrayTupleReference tuple;
+
+        public InsertionTask() {
+            this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS);
+            this.tuple = new ArrayTupleReference();
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            lock.lock();
+            try {
+                insertTaskStarted = true;
+
+                // insert tuples [101, 200]
+                insertIntTupleRange(101, 200);
+                condition.signal();
+                condition.await();
+
+                // insert tuples [1, 100]
+                insertIntTupleRange(1, 100);
+                condition.signal();
+                condition.await();
+
+                // insert tuples [201, 300] and delete tuple 151
+                insertIntTupleRange(201, 300);
+                TupleUtils.createIntegerTuple(builder, tuple, 151);
+                accessor.delete(tuple);
+                condition.signal();
+            } finally {
+                lock.unlock();
+            }
+
+            return true;
+        }
+
+        private void insertIntTupleRange(int begin, int end) throws Exception {
+            if (end < begin) {
+                throw new IllegalArgumentException("Invalid range: [" + begin + ", " + end + "]");
+            }
+
+            for (int i = begin; i <= end; i++) {
+                TupleUtils.createIntegerTuple(builder, tuple, i);
+                accessor.insert(tuple);
+            }
+        }
+
+    }
+
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
index 85bfdd2..6d7ee4d 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexBulkLoadTest.java
@@ -38,6 +38,8 @@
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         for (int i = 0; i < bulkLoadRounds; i++) {
             // We assume all fieldSerdes are of the same type. Check the first
             // one
@@ -55,9 +57,12 @@
                 orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
             }
         }
-        ctx.getIndex().close();
-    }
 
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
+    }
+    
     @Override
     protected String getTestOpName() {
         return "BulkLoad";
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java
index 93075a1..b96f252 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexDeleteTest.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexDeleteTest extends OrderedIndexTestDriver {
@@ -31,14 +32,16 @@
         this.orderedIndexTestUtils = new OrderedIndexTestUtils();
     }
 
-    private static final int numInsertRounds = 3;
-    private static final int numDeleteRounds = 3;
+    private static final int numInsertRounds = AccessMethodTestsConfig.BTREE_NUM_INSERT_ROUNDS;
+    private static final int numDeleteRounds = AccessMethodTestsConfig.BTREE_NUM_DELETE_ROUNDS;
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         for (int i = 0; i < numInsertRounds; i++) {
             // We assume all fieldSerdes are of the same type. Check the first
             // one to determine which field types to generate.
@@ -60,7 +63,10 @@
                 }
             }
         }
-        ctx.getIndex().close();
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index a29be89..970526e 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.hyracks.storage.am.btree;
 
+import static org.junit.Assert.fail;
+
 import java.util.Random;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -35,14 +37,16 @@
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.UnsortedInputException;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 @SuppressWarnings("rawtypes")
@@ -50,17 +54,13 @@
     protected static final Logger LOGGER = Logger.getLogger(OrderedIndexExamplesTest.class.getName());
     protected final Random rnd = new Random(50);
 
-    protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories)
-            throws TreeIndexException;
+    protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException;
 
-    protected abstract int getIndexFileId();
-    
     /**
-     * Fixed-Length Key,Value Example.
-     * 
-     * Create a tree index with one fixed-length key field and one fixed-length value
-     * field. Fill index with random values using insertions (not bulk load).
-     * Perform scans and range search.
+     * Fixed-Length Key,Value Example. Create a tree index with one fixed-length
+     * key field and one fixed-length value field. Fill index with random values
+     * using insertions (not bulk load). Perform scans and range search.
      */
     @Test
     public void fixedLengthKeyValueExample() throws Exception {
@@ -82,10 +82,13 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -93,7 +96,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
             int f0 = rnd.nextInt() % numInserts;
@@ -129,15 +133,92 @@
 
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Composite Key Example (Non-Unique Index).
-     * 
-     * Create a tree index with two fixed-length key fields and one fixed-length
-     * value field. Fill index with random values using insertions (not bulk
-     * load) Perform scans and range search.
+     * This test the btree page split. Originally this test didn't pass since
+     * the btree was spliting by cardinality and not size. Thus, we might end
+     * up with a situation where there is not enough space to insert the new
+     * tuple after the split which will throw an error and the split won't be
+     * propagated to upper level; thus, the tree is corrupted. Now, it split
+     * page by size. The correct behavior on abnormally large keys/values.
+     */
+    @Test
+    public void pageSplitTestExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree page split test.");
+        }
+
+        // Declare fields.
+        int fieldCount = 2;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
+        typeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
+        // Declare field serdes.
+        ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+
+        // Declare keys.
+        int keyFieldCount = 1;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+
+        String key = "111";
+        String data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "222";
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "333";
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "444";
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "555";
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        key = "666";
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, key, data);
+        indexAccessor.insert(tuple);
+
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
+    }
+
+    /**
+     * Composite Key Example (Non-Unique Index). Create a tree index with two
+     * fixed-length key fields and one fixed-length value field. Fill index with
+     * random values using insertions (not bulk load) Perform scans and range
+     * search.
      */
     @Test
     public void twoFixedLengthKeysOneFixedLengthValueExample() throws Exception {
@@ -161,10 +242,14 @@
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+        bloomFilterKeyFields[1] = 1;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -172,7 +257,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < 10000; i++) {
             int f0 = rnd.nextInt() % 2000;
@@ -210,7 +296,9 @@
         // Prefix-Range search in [-3, 3]
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
@@ -238,10 +326,13 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -249,7 +340,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         // Max string length to be generated.
         int maxLength = 10;
         int numInserts = 10000;
@@ -259,7 +351,7 @@
             TupleUtils.createTuple(tb, tuple, fieldSerdes, f0, f1);
             if (LOGGER.isLoggable(Level.INFO)) {
                 if (i % 1000 == 0) {
-                    LOGGER.info("Inserting " + f0 + " " + f1);
+                    LOGGER.info("Inserting[" + i + "] " + f0 + " " + f1);
                 }
             }
             try {
@@ -287,15 +379,16 @@
 
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Deletion Example.
-     * 
-     * Create a BTree with one variable-length key field and one variable-length
-     * value field. Fill B-tree with random values using insertions, then delete
-     * entries one-by-one. Repeat procedure a few times on same BTree.
+     * Deletion Example. Create a BTree with one variable-length key field and
+     * one variable-length value field. Fill B-tree with random values using
+     * insertions, then delete entries one-by-one. Repeat procedure a few times
+     * on same BTree.
      */
     @Test
     public void deleteExample() throws Exception {
@@ -317,14 +410,18 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         // Max string length to be generated.
         int runs = 3;
         for (int run = 0; run < runs; run++) {
@@ -388,15 +485,16 @@
                 break;
             }
         }
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Update example.
-     * 
-     * Create a BTree with one variable-length key field and one variable-length
-     * value field. Fill B-tree with random values using insertions, then update
-     * entries one-by-one. Repeat procedure a few times on same BTree.
+     * Update example. Create a BTree with one variable-length key field and one
+     * variable-length value field. Fill B-tree with random values using
+     * insertions, then update entries one-by-one. Repeat procedure a few times
+     * on same BTree.
      */
     @Test
     public void updateExample() throws Exception {
@@ -418,15 +516,19 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Inserting into tree...");
         }
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         int maxLength = 10;
@@ -474,15 +576,14 @@
             // Do another scan after a round of updates.
             orderedScan(indexAccessor, fieldSerdes);
         }
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Bulk load example.
-     * 
-     * Load a tree with 100,000 tuples. BTree has a composite key to "simulate"
-     * non-unique index creation.
-     * 
+     * Bulk load example. Load a tree with 100,000 tuples. BTree has a composite
+     * key to "simulate" non-unique index creation.
      */
     @Test
     public void bulkLoadExample() throws Exception {
@@ -505,10 +606,14 @@
         cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
         cmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+        bloomFilterKeyFields[1] = 1;
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        treeIndex.create();
+        treeIndex.activate();
 
         // Load sorted records.
         int ins = 100000;
@@ -516,20 +621,21 @@
             LOGGER.info("Bulk loading " + ins + " tuples");
         }
         long start = System.currentTimeMillis();
-        IIndexBulkLoadContext bulkLoadCtx = treeIndex.beginBulkLoad(0.7f);
+        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, ins);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         for (int i = 0; i < ins; i++) {
             TupleUtils.createIntegerTuple(tb, tuple, i, i, 5);
-            treeIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
+            bulkLoader.add(tuple);
         }
-        treeIndex.endBulkLoad(bulkLoadCtx);
+        bulkLoader.end();
         long end = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(ins + " tuples loaded in " + (end - start) + "ms");
         }
 
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
 
         // Build low key.
         ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(1);
@@ -544,15 +650,85 @@
         // Prefix-Range search in [44444, 44500]
         rangeSearch(cmpFactories, indexAccessor, fieldSerdes, lowKey, highKey);
 
-        treeIndex.close();
+        treeIndex.validate();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
-    private void orderedScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes)
-            throws Exception {
+    /**
+     * Bulk load failure example. Repeatedly loads a tree with 1,000 tuples, of
+     * which one tuple at each possible position does not conform to the
+     * expected order. We expect the bulk load to fail with an exception.
+     */
+    @Test
+    public void bulkOrderVerificationExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Bulk load order verification example");
+        }
+        // Declare fields.
+        int fieldCount = 2;
+        ITypeTraits[] typeTraits = new ITypeTraits[fieldCount];
+        typeTraits[0] = IntegerPointable.TYPE_TRAITS;
+        typeTraits[1] = IntegerPointable.TYPE_TRAITS;
+
+        // declare keys
+        int keyFieldCount = 1;
+        IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+        cmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        Random rnd = new Random();
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[keyFieldCount];
+        bloomFilterKeyFields[0] = 0;
+
+        int ins = 1000;
+        for (int i = 1; i < ins; i++) {
+            ITreeIndex treeIndex = createTreeIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+            treeIndex.create();
+            treeIndex.activate();
+
+            // Load sorted records, and expect to fail at tuple i.
+            IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, true, ins);
+            for (int j = 0; j < ins; j++) {
+                if (j > i) {
+                    fail("Bulk load failure test unexpectedly succeeded past tuple: " + j);
+                }
+                int key = j;
+                if (j == i) {
+                    int swapElementCase = Math.abs(rnd.nextInt()) % 2;
+                    if (swapElementCase == 0) {
+                        // Element equal to previous element.
+                        key--;
+                    } else {
+                        // Element smaller than previous element.
+                        key -= Math.abs(Math.random() % (ins - 1)) + 1;
+                    }
+                }
+                TupleUtils.createIntegerTuple(tb, tuple, key, 5);
+                try {
+                    bulkLoader.add(tuple);
+                } catch (UnsortedInputException e) {
+                    if (j != i) {
+                        fail("Unexpected exception: " + e.getMessage());
+                    }
+                    // Success.
+                    break;
+                }
+            }
+
+            treeIndex.deactivate();
+            treeIndex.destroy();
+        }
+    }
+
+    private void orderedScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Ordered Scan:");
         }
-        IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor();        
+        IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor();
         RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null);
         indexAccessor.search(scanCursor, nullPred);
         try {
@@ -569,45 +745,44 @@
         }
     }
 
-	private void diskOrderScan(IIndexAccessor indexAccessor,
-			ISerializerDeserializer[] fieldSerdes) throws Exception {
-		try {
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Disk-Order Scan:");
-			}
-			ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
-			TreeDiskOrderScanCursor diskOrderCursor = (TreeDiskOrderScanCursor) treeIndexAccessor
-					.createDiskOrderScanCursor();
-			treeIndexAccessor.diskOrderScan(diskOrderCursor);
-			try {
-				while (diskOrderCursor.hasNext()) {
-					diskOrderCursor.next();
-					ITupleReference frameTuple = diskOrderCursor.getTuple();
-					String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
-					if (LOGGER.isLoggable(Level.INFO)) {
-						LOGGER.info(rec);
-					}
-				}
-			} finally {
-				diskOrderCursor.close();
-			}
-		} catch (UnsupportedOperationException e) {
-			// Ignore exception because some indexes, e.g. the LSMBTree, don't
-			// support disk-order scan.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		} catch (ClassCastException e) {
-			// Ignore exception because IIndexAccessor sometimes isn't
-			// an ITreeIndexAccessor, e.g., for the LSMBTree.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		}
-	}
+    private void diskOrderScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
+        try {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Disk-Order Scan:");
+            }
+            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.isLoggable(Level.INFO)) {
+                        LOGGER.info(rec);
+                    }
+                }
+            } finally {
+                diskOrderCursor.close();
+            }
+        } catch (UnsupportedOperationException e) {
+            // Ignore exception because some indexes, e.g. the LSMBTree, don't
+            // support disk-order scan.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        } catch (ClassCastException e) {
+            // Ignore exception because IIndexAccessor sometimes isn't
+            // an ITreeIndexAccessor, e.g., for the LSMBTree.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        }
+    }
 
-    private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes,
-            ITupleReference lowKey, ITupleReference highKey) throws Exception {
+    private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor indexAccessor,
+            ISerializerDeserializer[] fieldSerdes, ITupleReference lowKey, ITupleReference highKey) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             String lowKeyString = TupleUtils.printTuple(lowKey, fieldSerdes);
             String highKeyString = TupleUtils.printTuple(highKey, fieldSerdes);
@@ -616,8 +791,7 @@
         ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
         MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, lowKey);
         MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, highKey);
-        RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp,
-                highKeySearchCmp);
+        RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp);
         indexAccessor.search(rangeCursor, rangePred);
         try {
             while (rangeCursor.hasNext()) {
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java
index d12603b..32b597c 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexInsertTest.java
@@ -24,12 +24,10 @@
 /**
  * Tests the BTree insert operation with strings and integer fields using
  * various numbers of key and payload fields.
- * 
  * Each tests first fills a BTree with randomly generated tuples. We compare the
  * following operations against expected results: 1. Point searches for all
  * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
  * search for composite keys).
- * 
  */
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexInsertTest extends OrderedIndexTestDriver {
@@ -46,6 +44,8 @@
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // We assume all fieldSerdes are of the same type. Check the first one
         // to determine which field types to generate.
         if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
@@ -62,7 +62,10 @@
         if (prefixLowKey != null && prefixHighKey != null) {
             orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
         }
-        ctx.getIndex().close();
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
index 3a894a2..fa22f6b 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexMultiThreadTest.java
@@ -29,95 +29,98 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IndexMultiThreadTestDriver;
 import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexMultiThreadTestDriver;
-import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
-public abstract class OrderedIndexMultiThreadTest {    
-    
+public abstract class OrderedIndexMultiThreadTest {
+
     protected final Logger LOGGER = Logger.getLogger(OrderedIndexMultiThreadTest.class.getName());
-    
+
     // Machine-specific number of threads to use for testing.
     protected final int REGULAR_NUM_THREADS = Runtime.getRuntime().availableProcessors();
     // Excessive number of threads for testing.
     protected final int EXCESSIVE_NUM_THREADS = Runtime.getRuntime().availableProcessors() * 4;
-    protected final int NUM_OPERATIONS = 10000;
-    
-    protected ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();    
-    
-    protected abstract void setUp() throws HyracksException;
-    
-    protected abstract void tearDown() throws HyracksDataException;        
+    protected final int NUM_OPERATIONS = AccessMethodTestsConfig.BTREE_MULTITHREAD_NUM_OPERATIONS;
 
-    protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories) throws TreeIndexException;
-    
-    protected abstract int getFileId();
-    
-    protected abstract ITreeIndexTestWorkerFactory getWorkerFactory();
-    
+    protected ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
+
+    protected abstract void setUp() throws HyracksException;
+
+    protected abstract void tearDown() throws HyracksDataException;
+
+    protected abstract IIndex createIndex(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields) throws TreeIndexException;
+
+    protected abstract IIndexTestWorkerFactory getWorkerFactory();
+
     protected abstract ArrayList<TestWorkloadConf> getTestWorkloadConf();
-    
+
     protected abstract String getIndexTypeName();
-    
-    protected static float[] getUniformOpProbs(TestOperation[] ops) {
-        float[] opProbs = new float[ops.length];
-        for (int i = 0; i < ops.length; i++) {
-            opProbs[i] = 1.0f / (float) ops.length;
-        }
-        return opProbs;
-    }
-    
-    protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, int numThreads, TestWorkloadConf conf, String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
+
+    protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, int numThreads, TestWorkloadConf conf,
+            String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
         setUp();
-        
+
         if (LOGGER.isLoggable(Level.INFO)) {
-        	String indexTypeName = getIndexTypeName();
-            LOGGER.info(indexTypeName + " MultiThread Test:\nData: " + dataMsg + "; Threads: " + numThreads + "; Workload: " + conf.toString() + ".");
+            String indexTypeName = getIndexTypeName();
+            LOGGER.info(indexTypeName + " MultiThread Test:\nData: " + dataMsg + "; Threads: " + numThreads
+                    + "; Workload: " + conf.toString() + ".");
         }
-        
+
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
-        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeys);     
-        
-        ITreeIndex index = createTreeIndex(typeTraits, cmpFactories);
-        ITreeIndexTestWorkerFactory workerFactory = getWorkerFactory();
-        
+        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeys);
+
+        // This is only used for the LSM-BTree.
+        int[] bloomFilterKeyFields = new int[numKeys];
+        for (int i = 0; i < numKeys; ++i) {
+            bloomFilterKeyFields[i] = i;
+        }
+
+        IIndex index = createIndex(typeTraits, cmpFactories, bloomFilterKeyFields);
+        IIndexTestWorkerFactory workerFactory = getWorkerFactory();
+
         // 4 batches per thread.
         int batchSize = (NUM_OPERATIONS / numThreads) / 4;
-        
-        TreeIndexMultiThreadTestDriver driver = new TreeIndexMultiThreadTestDriver(index, workerFactory, fieldSerdes, conf.ops, conf.opProbs);
-        driver.init(getFileId());
+
+        IndexMultiThreadTestDriver driver = new IndexMultiThreadTestDriver(index, workerFactory, fieldSerdes, conf.ops,
+                conf.opProbs);
+        driver.init();
         long[] times = driver.run(numThreads, 1, NUM_OPERATIONS, batchSize);
+        index.validate();
         driver.deinit();
-        
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("BTree MultiThread Test Time: " + times[0] + "ms");
         }
-        
+
         tearDown();
     }
-    
+
     @Test
-    public void oneIntKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {        
-        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+    public void oneIntKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE };
         int numKeys = 1;
         String dataMsg = "One Int Key And Value";
-        
+
         for (TestWorkloadConf conf : workloadConfs) {
             runTest(fieldSerdes, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
             runTest(fieldSerdes, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
         }
     }
-    
+
     @Test
-    public void oneStringKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {        
-        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+    public void oneStringKeyAndValue() throws InterruptedException, TreeIndexException, HyracksException {
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
         int numKeys = 1;
         String dataMsg = "One String Key And Value";
-        
+
         for (TestWorkloadConf conf : workloadConfs) {
             runTest(fieldSerdes, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
             runTest(fieldSerdes, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
index f75a1f1..4ab0e87 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestContext.java
@@ -20,16 +20,16 @@
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 @SuppressWarnings("rawtypes")
-public abstract class OrderedIndexTestContext extends TreeIndexTestContext<CheckTuple> {
+public abstract class OrderedIndexTestContext extends IndexTestContext<CheckTuple> {
 
     protected final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
 
-    public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
-        super(fieldSerdes, treeIndex);
+    public OrderedIndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
+        super(fieldSerdes, index);
     }
 
     public void upsertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java
index 8daa5e0..ef2ee0b 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestDriver.java
@@ -27,12 +27,13 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexTestDriver {
     protected final Logger LOGGER = Logger.getLogger(OrderedIndexTestDriver.class.getName());
 
-    protected static final int numTuplesToInsert = 10000;
+    protected static final int numTuplesToInsert = AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT;
 
     protected abstract OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
             BTreeLeafFrameType leafType) throws Exception;
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index a053dde..04c64fe 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -7,8 +7,8 @@
 import java.io.DataInputStream;
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.NavigableSet;
 import java.util.Random;
+import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -25,7 +25,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
 import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
@@ -43,47 +43,36 @@
                     actual.getFieldLength(i));
             DataInput dataIn = new DataInputStream(inStream);
             Object actualObj = fieldSerdes[i].deserialize(dataIn);
-            if (!actualObj.equals(expected.get(i))) {
-                fail("Actual and expected fields do not match on field " + i + ".\nExpected: " + expected.get(i)
+            if (!actualObj.equals(expected.getField(i))) {
+                fail("Actual and expected fields do not match on field " + i + ".\nExpected: " + expected.getField(i)
                         + "\nActual  : " + actualObj);
             }
         }
     }
 
     @SuppressWarnings("unchecked")
-    // Create a new TreeSet containing the elements satisfying the prefix
-    // search.
+    // Create a new TreeSet containing the elements satisfying the prefix search.
     // Implementing prefix search by changing compareTo() in CheckTuple does not
     // work.
-    public static TreeSet<CheckTuple> getPrefixExpectedSubset(TreeSet<CheckTuple> checkTuples, CheckTuple lowKey,
+    public static SortedSet<CheckTuple> getPrefixExpectedSubset(TreeSet<CheckTuple> checkTuples, CheckTuple lowKey,
             CheckTuple highKey) {
-        TreeSet<CheckTuple> expectedSubset = new TreeSet<CheckTuple>();
-        Iterator<CheckTuple> iter = checkTuples.iterator();
-        while (iter.hasNext()) {
-            CheckTuple t = iter.next();
-            boolean geLowKey = true;
-            boolean leHighKey = true;
-            for (int i = 0; i < lowKey.getNumKeys(); i++) {
-                if (t.get(i).compareTo(lowKey.get(i)) < 0) {
-                    geLowKey = false;
-                    break;
-                }
-            }
-            for (int i = 0; i < highKey.getNumKeys(); i++) {
-                if (t.get(i).compareTo(highKey.get(i)) > 0) {
-                    leHighKey = false;
-                    break;
-                }
-            }
-            if (geLowKey && leHighKey) {
-                expectedSubset.add(t);
-            }
+        lowKey.setIsHighKey(false);
+        highKey.setIsHighKey(true);
+        CheckTuple low = checkTuples.ceiling(lowKey);
+        CheckTuple high = checkTuples.floor(highKey);
+        if (low == null || high == null) {
+            // Must be empty.
+            return new TreeSet<CheckTuple>();
         }
-        return expectedSubset;
+        if (high.compareTo(low) < 0) {
+            // Must be empty.
+            return new TreeSet<CheckTuple>();
+        }
+        return checkTuples.subSet(low, true, high, true);
     }
 
     @SuppressWarnings("unchecked")
-    public void checkRangeSearch(ITreeIndexTestContext ctx, ITupleReference lowKey, ITupleReference highKey,
+    public void checkRangeSearch(IIndexTestContext ctx, ITupleReference lowKey, ITupleReference highKey,
             boolean lowKeyInclusive, boolean highKeyInclusive) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Range Search.");
@@ -99,7 +88,7 @@
         CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, ctx.getFieldSerdes(), lowKeyCmp.getKeyFieldCount());
         CheckTuple highKeyCheck = createCheckTupleFromTuple(highKey, ctx.getFieldSerdes(),
                 highKeyCmp.getKeyFieldCount());
-        NavigableSet<CheckTuple> expectedSubset = null;
+        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).
@@ -132,7 +121,7 @@
         }
     }
 
-    public void checkPointSearches(ITreeIndexTestContext ictx) throws Exception {
+    public void checkPointSearches(IIndexTestContext ictx) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Point Searches On All Expected Keys.");
         }
@@ -177,7 +166,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    public void insertStringTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         String[] fieldValues = new String[fieldCount];
@@ -207,10 +196,10 @@
             }
         }
     }
-    
-    public void upsertStringTuples(ITreeIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
-    	OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
-    	int fieldCount = ctx.getFieldCount();
+
+    public void upsertStringTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
+        OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
+        int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         String[] fieldValues = new String[fieldCount];
         for (int i = 0; i < numTuples; i++) {
@@ -235,7 +224,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    public void bulkLoadStringTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void bulkLoadStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         String[] fieldValues = new String[fieldCount];
@@ -262,9 +251,9 @@
         }
     }
 
-    public void upsertIntTuples(ITreeIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
+    public void upsertIntTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
         OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
-    	int fieldCount = ctx.getFieldCount();
+        int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
         // Scale range of values according to number of keys.
@@ -286,9 +275,9 @@
             ctx.upsertCheckTuple(createIntCheckTuple(fieldValues, ctx.getKeyFieldCount()), ctx.getCheckTuples());
         }
     }
-    
+
     @SuppressWarnings("unchecked")
-    public void updateTuples(ITreeIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
+    public void updateTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
         OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
         int fieldCount = ctx.getFieldCount();
         int keyFieldCount = ctx.getKeyFieldCount();
@@ -317,7 +306,7 @@
             // Update check tuple's non-key fields.
             for (int j = keyFieldCount; j < fieldCount; j++) {
                 Comparable newValue = getRandomUpdateValue(ctx.getFieldSerdes()[j], rnd);
-                checkTuple.set(j, newValue);
+                checkTuple.setField(j, newValue);
             }
 
             createTupleFromCheckTuple(checkTuple, updateTupleBuilder, updateTuple, ctx.getFieldSerdes());
@@ -334,7 +323,7 @@
     public CheckTuple createStringCheckTuple(String[] fieldValues, int numKeyFields) {
         CheckTuple<String> checkTuple = new CheckTuple<String>(fieldValues.length, numKeyFields);
         for (String s : fieldValues) {
-            checkTuple.add((String) s);
+            checkTuple.appendField((String) s);
         }
         return checkTuple;
     }
@@ -396,7 +385,7 @@
     protected CheckTuple createIntCheckTuple(int[] fieldValues, int numKeyFields) {
         CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldValues.length, numKeyFields);
         for (int v : fieldValues) {
-            checkTuple.add(v);
+            checkTuple.appendField(v);
         }
         return checkTuple;
     }
@@ -421,12 +410,12 @@
     }
 
     @Override
-    protected ArrayTupleBuilder createDeleteTupleBuilder(ITreeIndexTestContext ctx) {
+    protected ArrayTupleBuilder createDeleteTupleBuilder(IIndexTestContext ctx) {
         return new ArrayTupleBuilder(ctx.getKeyFieldCount());
     }
 
     @Override
-    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, ITreeIndexTestContext ctx)
+    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, IIndexTestContext ctx)
             throws HyracksDataException {
         @SuppressWarnings("unchecked")
         TreeSet<CheckTuple> checkTuples = (TreeSet<CheckTuple>) ctx.getCheckTuples();
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java
index 65b2ade..049724e 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpdateTest.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
 
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexUpdateTest extends OrderedIndexTestDriver {
@@ -31,7 +32,7 @@
         this.orderedIndexTestUtils = new OrderedIndexTestUtils();
     }
 
-    private static final int numUpdateRounds = 3;
+    private static final int numUpdateRounds = AccessMethodTestsConfig.BTREE_NUM_UPDATE_ROUNDS;
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType,
@@ -42,6 +43,8 @@
             return;
         }
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // We assume all fieldSerdes are of the same type. Check the first one
         // to determine which field types to generate.
         if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
@@ -60,6 +63,10 @@
                 orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
             }
         }
+
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java
index 0d94a18..d34928f 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexUpsertTest.java
@@ -24,12 +24,10 @@
 /**
  * Tests the BTree insert operation with strings and integer fields using
  * various numbers of key and payload fields.
- * 
  * Each tests first fills a BTree with randomly generated tuples. We compare the
  * following operations against expected results: 1. Point searches for all
  * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix
  * search for composite keys).
- * 
  */
 @SuppressWarnings("rawtypes")
 public abstract class OrderedIndexUpsertTest extends OrderedIndexTestDriver {
@@ -46,6 +44,8 @@
             ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey)
             throws Exception {
         OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // We assume all fieldSerdes are of the same type. Check the first one
         // to determine which field types to generate.
         if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
@@ -62,7 +62,9 @@
         if (prefixLowKey != null && prefixHighKey != null) {
             orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true);
         }
-        ctx.getIndex().close();
+        ctx.getIndex().validate();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
new file mode 100644
index 0000000..2226d94
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexLifecycleTest.java
@@ -0,0 +1,96 @@
+package edu.uci.ics.hyracks.storage.am.common;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+
+public abstract class AbstractIndexLifecycleTest {
+
+    protected IIndex index;
+
+    protected abstract boolean persistentStateExists() throws Exception;
+
+    protected abstract boolean isEmptyIndex() throws Exception;
+
+    protected abstract void performInsertions() throws Exception;
+
+    protected abstract void checkInsertions() throws Exception;
+
+    protected abstract void clearCheckableInsertions() throws Exception;
+
+    @Before
+    public abstract void setup() throws Exception;
+
+    @After
+    public abstract void tearDown() throws Exception;
+
+    @Test
+    public void validSequenceTest() throws Exception {
+        // Double create is valid
+        index.create();
+        Assert.assertTrue(persistentStateExists());
+        index.create();
+        Assert.assertTrue(persistentStateExists());
+
+        // Double open is valid
+        index.activate();
+        index.activate();
+        Assert.assertTrue(isEmptyIndex());
+
+        // Insert some stuff
+        performInsertions();
+        checkInsertions();
+
+        // Check that the inserted stuff isn't there
+        clearCheckableInsertions();
+        index.clear();
+        Assert.assertTrue(isEmptyIndex());
+
+        // Insert more stuff
+        performInsertions();
+
+        // Double close is valid
+        index.deactivate();
+        index.deactivate();
+
+        // Check that the inserted stuff is still there
+        index.activate();
+        checkInsertions();
+        index.deactivate();
+
+        // Double destroy is valid
+        index.destroy();
+        Assert.assertFalse(persistentStateExists());
+        index.destroy();
+        Assert.assertFalse(persistentStateExists());
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest1() throws Exception {
+        index.create();
+        index.activate();
+        index.create();
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest2() throws Exception {
+        index.create();
+        index.activate();
+        index.destroy();
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest3() throws Exception {
+        index.create();
+        index.clear();
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void invalidSequenceTest4() throws Exception {
+        index.clear();
+    }
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractTreeIndexTestWorker.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
similarity index 78%
rename from fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractTreeIndexTestWorker.java
rename to fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
index eca9b35..f9ff26a 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractTreeIndexTestWorker.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/AbstractIndexTestWorker.java
@@ -20,32 +20,34 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
 import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
 
-public abstract class AbstractTreeIndexTestWorker extends Thread implements ITreeIndexTestWorker {
-    private Random rnd = new Random();
+public abstract class AbstractIndexTestWorker extends Thread implements ITreeIndexTestWorker {
+    private final Random rnd;
     private final DataGenThread dataGen;
     private final TestOperationSelector opSelector;
     private final int numBatches;
-    
+
     protected final IIndexAccessor indexAccessor;
-    
-    public AbstractTreeIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches) {
+
+    public AbstractIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index, int numBatches) {
         this.dataGen = dataGen;
         this.opSelector = opSelector;
         this.numBatches = numBatches;
-        indexAccessor = index.createAccessor();
+        this.rnd = new Random();
+        this.indexAccessor = index.createAccessor(TestOperationCallback.INSTANCE, TestOperationCallback.INSTANCE);
     }
-    
+
     @Override
     public void run() {
         try {
             for (int i = 0; i < numBatches; i++) {
-                TupleBatch batch = dataGen.getBatch();     
+                TupleBatch batch = dataGen.getBatch();
                 for (int j = 0; j < batch.size(); j++) {
                     TestOperation op = opSelector.getOp(rnd.nextInt());
                     ITupleReference tuple = batch.get(j);
@@ -57,8 +59,8 @@
             e.printStackTrace();
         }
     }
-    
-    protected void consumeCursorTuples(IIndexCursor cursor) throws HyracksDataException {
+
+    protected void consumeCursorTuples(IIndexCursor cursor) throws HyracksDataException, IndexException {
         try {
             while (cursor.hasNext()) {
                 cursor.next();
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
index 4b4b90b..b7037d6 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
@@ -17,42 +17,75 @@
 
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class CheckTuple<T extends Comparable<T>> implements Comparable<T> {
-    protected final int numKeys;    
-    protected final Comparable[] tuple;
+    protected final int numKeys;
+    protected final Comparable[] fields;
     protected int pos;
+    protected boolean isHighKey;
 
     public CheckTuple(int numFields, int numKeys) {
         this.numKeys = numKeys;
-        this.tuple = new Comparable[numFields];
+        this.fields = new Comparable[numFields];
         pos = 0;
+        isHighKey = false;
     }
 
-    public void add(T e) {
-        tuple[pos++] = e;
+    public void appendField(T e) {
+        fields[pos++] = e;
     }
 
-    @Override
-    public int compareTo(T o) {
-        CheckTuple<T> other = (CheckTuple<T>)o;
-        for (int i = 0; i < numKeys; i++) {            
-            int cmp = tuple[i].compareTo(other.get(i));
-            if (cmp != 0) {
-                return cmp;
-            }
+	@Override
+	public int compareTo(T o) {
+		CheckTuple<T> other = (CheckTuple<T>) o;
+		int cmpFieldCount = Math.min(other.getNumKeys(), numKeys);
+		for (int i = 0; i < cmpFieldCount; i++) {
+			int cmp = fields[i].compareTo(other.getField(i));
+			if (cmp != 0) {
+				return cmp;
+			}
+		}
+		if (other.getNumKeys() == numKeys) {
+		    return 0;
+		}
+		if (other.getNumKeys() < numKeys) {
+		    return (other.isHighKey) ? -1 : 1;
+		}
+		if (other.getNumKeys() > numKeys) {
+            return (isHighKey) ? 1 : -1;
         }
-        return 0;
-    }
+		return 0;
+	}
 
-    public T get(int idx) {
-        return (T)tuple[idx];
-    }
+	@Override
+	public boolean equals(Object o) {
+		if (!(o instanceof Comparable<?>)) {
+			return false;
+		}
+		return compareTo((T) o) == 0;
+	}
     
-    public void set(int idx, T e) {
-        tuple[idx] = e;
+	@Override
+	public int hashCode() {
+		int hash = 0;
+		for (int i = 0; i < numKeys; i++) {
+			hash = 37 * hash + fields[i].hashCode();
+		}
+		return hash;
+	}
+	
+	public void setIsHighKey(boolean isHighKey) {
+	    this.isHighKey = isHighKey;
+	}
+	
+	public T getField(int idx) {
+		return (T) fields[idx];
+	}
+    
+    public void setField(int idx, T e) {
+        fields[idx] = e;
     }
     
     public int size() {
-        return tuple.length;
+        return fields.length;
     }
     
     public int getNumKeys() {
@@ -62,9 +95,9 @@
     @Override
     public String toString() {
         StringBuilder strBuilder = new StringBuilder();
-        for (int i = 0; i < tuple.length; i++) {
-            strBuilder.append(tuple[i].toString());
-            if (i != tuple.length-1) {
+        for (int i = 0; i < fields.length; i++) {
+            strBuilder.append(fields[i].toString());
+            if (i != fields.length-1) {
                 strBuilder.append(" ");
             }
         }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestContext.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestContext.java
new file mode 100644
index 0000000..3599c5e
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestContext.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.storage.am.common;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+
+@SuppressWarnings("rawtypes")
+public interface IIndexTestContext<T extends CheckTuple> {
+    public int getFieldCount();
+
+    public int getKeyFieldCount();
+
+    public ISerializerDeserializer[] getFieldSerdes();
+
+    public IBinaryComparatorFactory[] getComparatorFactories();
+
+    public IIndexAccessor getIndexAccessor();
+
+    public IIndex getIndex();
+
+    public ArrayTupleReference getTuple();
+
+    public ArrayTupleBuilder getTupleBuilder();
+
+    public void insertCheckTuple(T checkTuple, Collection<T> checkTuples);      
+
+    public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples);
+
+    public Collection<T> getCheckTuples();
+
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/ITreeIndexTestWorkerFactory.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
similarity index 75%
rename from fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/ITreeIndexTestWorkerFactory.java
rename to fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
index 64b5aea..d4efb3e 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/ITreeIndexTestWorkerFactory.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IIndexTestWorkerFactory.java
@@ -15,9 +15,10 @@
 
 package edu.uci.ics.hyracks.storage.am.common;
 
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
 
-public interface ITreeIndexTestWorkerFactory {
-    public AbstractTreeIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches);
+public interface IIndexTestWorkerFactory {
+    public AbstractIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, IIndex index,
+            int numBatches);
 }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexMultiThreadTestDriver.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
similarity index 74%
rename from fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexMultiThreadTestDriver.java
rename to fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
index 8c1d06f..ca1d28f 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexMultiThreadTestDriver.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexMultiThreadTestDriver.java
@@ -18,34 +18,35 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
 
 @SuppressWarnings("rawtypes")
-public class TreeIndexMultiThreadTestDriver {
-    private static final int RANDOM_SEED = 50;
+public class IndexMultiThreadTestDriver {
+    protected static final int RANDOM_SEED = 50;
     // Means no additional payload. Only the specified fields.
-    private static final int PAYLOAD_SIZE = 0;
-    private final TestOperationSelector opSelector;    
-    private final ISerializerDeserializer[] fieldSerdes;
-    private final ITreeIndex index;
-    private final ITreeIndexTestWorkerFactory workerFactory;
-    
-    public TreeIndexMultiThreadTestDriver(ITreeIndex index, ITreeIndexTestWorkerFactory workerFactory,
-            ISerializerDeserializer[] fieldSerdes, TestOperation[] ops, float[] opProbs) {
+    protected static final int PAYLOAD_SIZE = 0;
+    protected final TestOperationSelector opSelector;
+    protected final ISerializerDeserializer[] fieldSerdes;
+    protected final IIndex index;
+    protected final IIndexTestWorkerFactory workerFactory;
+
+    public IndexMultiThreadTestDriver(IIndex index, IIndexTestWorkerFactory workerFactory,
+            ISerializerDeserializer[] fieldSerdes, TestOperation[] ops, double[] opProbs) {
         this.index = index;
         this.workerFactory = workerFactory;
         this.fieldSerdes = fieldSerdes;
         this.opSelector = new TestOperationSelector(ops, opProbs);
-    }      
-    
-    public void init(int fileId) throws HyracksDataException {
-    	index.create(fileId);
-    	index.open(fileId);
     }
-    
-    public long[] run(int numThreads, int numRepeats, int numOps, int batchSize) throws InterruptedException, TreeIndexException {
+
+    public void init() throws HyracksDataException {
+        index.create();
+        index.activate();
+    }
+
+    public long[] run(int numThreads, int numRepeats, int numOps, int batchSize) throws InterruptedException,
+            TreeIndexException {
         int numBatches = numOps / batchSize;
         int threadNumBatches = numBatches / numThreads;
         if (threadNumBatches <= 0) {
@@ -59,16 +60,16 @@
             while (dataGen.tupleBatchQueue.remainingCapacity() != 0 && dataGen.tupleBatchQueue.size() != numBatches) {
                 Thread.sleep(10);
             }
-                        
+
             // Start worker threads.
-            AbstractTreeIndexTestWorker[] workers = new AbstractTreeIndexTestWorker[numThreads];
+            AbstractIndexTestWorker[] workers = new AbstractIndexTestWorker[numThreads];
             long start = System.currentTimeMillis();
             for (int j = 0; j < numThreads; j++) {
                 workers[j] = workerFactory.create(dataGen, opSelector, index, threadNumBatches);
                 workers[j].start();
             }
             // Join worker threads.
-            for (int j = 0; j < numThreads; j++) {                
+            for (int j = 0; j < numThreads; j++) {
                 workers[j].join();
             }
             long end = System.currentTimeMillis();
@@ -76,13 +77,15 @@
         }
         return times;
     }
-    
+
     public void deinit() throws HyracksDataException {
-    	index.close();
+        index.deactivate();
+        index.destroy();
     }
-    
+
     // To allow subclasses to override the data gen params.
     public DataGenThread createDatagenThread(int numThreads, int numBatches, int batchSize) {
-        return new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, PAYLOAD_SIZE, RANDOM_SEED, 2*numThreads, false);
+        return new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, PAYLOAD_SIZE, RANDOM_SEED,
+                2 * numThreads, false);
     }
 }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestContext.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
similarity index 81%
rename from fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestContext.java
rename to fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
index bc5312c..e0aa1db 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestContext.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/IndexTestContext.java
@@ -20,21 +20,22 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 
 @SuppressWarnings("rawtypes")
-public abstract class TreeIndexTestContext<T extends CheckTuple> implements ITreeIndexTestContext<T> {
+public abstract class IndexTestContext<T extends CheckTuple> implements IIndexTestContext<T> {
     protected final ISerializerDeserializer[] fieldSerdes;
-    protected final ITreeIndex treeIndex;
+    protected final IIndex index;
     protected final ArrayTupleBuilder tupleBuilder;
     protected final ArrayTupleReference tuple = new ArrayTupleReference();
     protected final IIndexAccessor indexAccessor;
 
-    public TreeIndexTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
+    public IndexTestContext(ISerializerDeserializer[] fieldSerdes, IIndex index) {
         this.fieldSerdes = fieldSerdes;
-        this.treeIndex = treeIndex;
-        this.indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        this.index = index;
+        this.indexAccessor = (IIndexAccessor) index.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
         this.tupleBuilder = new ArrayTupleBuilder(fieldSerdes.length);
     }
 
@@ -64,8 +65,8 @@
     }
 
     @Override
-    public ITreeIndex getIndex() {
-        return treeIndex;
+    public IIndex getIndex() {
+        return index;
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationCallback.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationCallback.java
new file mode 100644
index 0000000..04f888b
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationCallback.java
@@ -0,0 +1,45 @@
+package edu.uci.ics.hyracks.storage.am.common;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+public enum TestOperationCallback implements ISearchOperationCallback, IModificationOperationCallback {
+    INSTANCE;
+
+    private static final int RANDOM_SEED = 50;
+    private final Random random = new Random();
+
+    private TestOperationCallback() {
+        random.setSeed(RANDOM_SEED);
+    }
+
+    @Override
+    public boolean proceed(ITupleReference tuple) {
+        // Always fail
+        return false;
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) {
+        // Do nothing.
+    }
+
+    @Override
+    public void before(ITupleReference tuple) {
+        // Do nothing.        
+    }
+
+    @Override
+    public void found(ITupleReference before, ITupleReference after) {
+        // Do nothing.        
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) {
+        // Do nothing.
+    }
+
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java
index 1ae79a1..47735e5 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestOperationSelector.java
@@ -15,7 +15,8 @@
 
 package edu.uci.ics.hyracks.storage.am.common;
 
-import java.util.Arrays;
+import edu.uci.ics.hyracks.storage.am.common.datagen.ProbabilityHelper;
+
 
 public class TestOperationSelector {
 
@@ -32,15 +33,15 @@
     }
     
     private final TestOperation[] ops;
-    private final int[] opRanges;    
+    private final int[] cumulIntRanges;    
     
-    public TestOperationSelector(TestOperation[] ops, float[] opProbs) {
+    public TestOperationSelector(TestOperation[] ops, double[] opProbs) {
         sanityCheck(ops, opProbs);
         this.ops = ops;
-        this.opRanges = getOpRanges(opProbs);
+        this.cumulIntRanges = ProbabilityHelper.getCumulIntRanges(opProbs);
     }
     
-    private void sanityCheck(TestOperation[] ops, float[] opProbs) {
+    private void sanityCheck(TestOperation[] ops, double[] opProbs) {
         if (ops.length == 0) {
             throw new RuntimeException("Empty op array.");
         }
@@ -59,25 +60,8 @@
         }
     }
     
-    private int[] getOpRanges(float[] opProbabilities) {
-        int[] opRanges = new int[opProbabilities.length];
-        if (opRanges.length > 1) {
-            opRanges[0] = (int) Math.floor(Integer.MAX_VALUE * opProbabilities[0]);
-            for (int i = 1; i < opRanges.length - 1; i++) {
-                opRanges[i] = opRanges[i - 1] + (int) Math.floor(Integer.MAX_VALUE * opProbabilities[i]);
-            }
-            opRanges[opRanges.length - 1] = Integer.MAX_VALUE;
-        } else {
-            opRanges[0] = Integer.MAX_VALUE;
-        }
-        return opRanges;
-    }
-    
     public TestOperation getOp(int randomInt) {
-        int ix = Arrays.binarySearch(opRanges, randomInt);
-        if (ix < 0) {
-            ix = -ix - 1;
-        }
+        int ix = ProbabilityHelper.choose(cumulIntRanges, randomInt);
         return ops[ix];
     }
 }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java
index 2437514..28845ed 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TestWorkloadConf.java
@@ -19,9 +19,9 @@
 
 public class TestWorkloadConf {
     public final TestOperation[] ops;
-    public final float[] opProbs;
+    public final double[] opProbs;
 
-    public TestWorkloadConf(TestOperation[] ops, float[] opProbs) {
+    public TestWorkloadConf(TestOperation[] ops, double[] opProbs) {
         this.ops = ops;
         this.opProbs = opProbs;
     }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
index d16553a..1a80231 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.storage.am.common;
 
 import static org.junit.Assert.fail;
@@ -18,7 +33,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
@@ -44,11 +59,11 @@
 
     protected abstract Collection createCheckTuplesCollection();
 
-    protected abstract ArrayTupleBuilder createDeleteTupleBuilder(ITreeIndexTestContext ctx);
+    protected abstract ArrayTupleBuilder createDeleteTupleBuilder(IIndexTestContext ctx);
 
     // See if tuple with corresponding checkTuple exists in ctx.checkTuples.
     protected abstract boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple,
-            ITreeIndexTestContext ctx) throws HyracksDataException;
+            IIndexTestContext ctx) throws HyracksDataException;
 
     @SuppressWarnings("unchecked")
     public static void createTupleFromCheckTuple(CheckTuple checkTuple, ArrayTupleBuilder tupleBuilder,
@@ -57,7 +72,7 @@
         DataOutput dos = tupleBuilder.getDataOutput();
         tupleBuilder.reset();
         for (int i = 0; i < fieldCount; i++) {
-            fieldSerdes[i].serialize(checkTuple.get(i), dos);
+            fieldSerdes[i].serialize(checkTuple.getField(i), dos);
             tupleBuilder.addFieldEndOffset();
         }
         tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
@@ -73,13 +88,13 @@
                     tuple.getFieldLength(i));
             DataInput dataIn = new DataInputStream(inStream);
             Comparable fieldObj = (Comparable) fieldSerdes[i].deserialize(dataIn);
-            checkTuple.add(fieldObj);
+            checkTuple.appendField(fieldObj);
         }
         return checkTuple;
     }
 
     @SuppressWarnings("unchecked")
-    public void checkScan(ITreeIndexTestContext ctx) throws Exception {
+    public void checkScan(IIndexTestContext ctx) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Scan.");
         }
@@ -90,7 +105,7 @@
         checkExpectedResults(scanCursor, ctx.getCheckTuples(), ctx.getFieldSerdes(), ctx.getKeyFieldCount(), checkIter);
     }
 
-    public void checkDiskOrderScan(ITreeIndexTestContext ctx) throws Exception {
+    public void checkDiskOrderScan(IIndexTestContext ctx) throws Exception {
         try {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Testing Disk-Order Scan.");
@@ -128,16 +143,16 @@
                 LOGGER.info("Ignoring disk-order scan since it's not supported.");
             }
         } catch (ClassCastException e) {
-			// Ignore exception because IIndexAccessor sometimes isn't
-			// an ITreeIndexAccessor, e.g., for the LSMBTree.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		}
+            // Ignore exception because IIndexAccessor sometimes isn't
+            // an ITreeIndexAccessor, e.g., for the LSMBTree.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        }
     }
 
     @SuppressWarnings("unchecked")
-    public void insertIntTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
@@ -165,9 +180,9 @@
             }
         }
     }
-    
+
     @SuppressWarnings("unchecked")
-    public void upsertIntTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void upsertIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
@@ -197,7 +212,7 @@
     }
 
     @SuppressWarnings("unchecked")
-    public void bulkLoadIntTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void bulkLoadIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         int[] fieldValues = new int[ctx.getFieldCount()];
@@ -221,14 +236,14 @@
         }
     }
 
-    public static void bulkLoadCheckTuples(ITreeIndexTestContext ctx, Collection<CheckTuple> checkTuples)
+    public static void bulkLoadCheckTuples(IIndexTestContext ctx, Collection<CheckTuple> checkTuples)
             throws HyracksDataException, IndexException {
         int fieldCount = ctx.getFieldCount();
         int numTuples = checkTuples.size();
         ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         // Perform bulk load.
-        IIndexBulkLoadContext bulkLoadCtx = ctx.getIndex().beginBulkLoad(0.7f);
+        IIndexBulkLoader bulkLoader = ctx.getIndex().createBulkLoader(0.7f, false, numTuples);
         int c = 1;
         for (CheckTuple checkTuple : checkTuples) {
             if (LOGGER.isLoggable(Level.INFO)) {
@@ -237,14 +252,14 @@
                 }
             }
             createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, ctx.getFieldSerdes());
-            ctx.getIndex().bulkLoadAddTuple(tuple, bulkLoadCtx);
+            bulkLoader.add(tuple);
             c++;
         }
-        ctx.getIndex().endBulkLoad(bulkLoadCtx);
+        bulkLoader.end();
     }
 
     @SuppressWarnings("unchecked")
-    public void deleteTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void deleteTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         ArrayTupleBuilder deleteTupleBuilder = createDeleteTupleBuilder(ctx);
         ArrayTupleReference deleteTuple = new ArrayTupleReference();
         int numCheckTuples = ctx.getCheckTuples().size();
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
new file mode 100644
index 0000000..f962200
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.storage.am.config;
+
+/**
+ * Global parameters used for executing access method JUnit tests.
+ */
+public class AccessMethodTestsConfig {
+    // Test params for RTree, LSMRTree and LSMRTreeWithAntiMatterTuples.
+    public static final int RTREE_NUM_TUPLES_TO_INSERT = 100;
+    public static final int RTREE_NUM_INSERT_ROUNDS = 2;
+    public static final int RTREE_NUM_DELETE_ROUNDS = 2;
+    public static final int RTREE_MULTITHREAD_NUM_OPERATIONS = 200;
+    public static final boolean RTREE_TEST_RSTAR_POLICY = true;
+    // Test params for LSMRTree and LSMRTreeWithAntiMatterTuples.
+    public static final int LSM_RTREE_BULKLOAD_ROUNDS = 5;
+    public static final int LSM_RTREE_MAX_TREES_TO_MERGE = 3;
+    public static final boolean LSM_RTREE_TEST_RSTAR_POLICY = false;
+
+    // Test params for BTree, LSMBTree.
+    public static final int BTREE_NUM_TUPLES_TO_INSERT = 100;
+    public static final int BTREE_NUM_INSERT_ROUNDS = 3;
+    public static final int BTREE_NUM_DELETE_ROUNDS = 3;
+    public static final int BTREE_NUM_UPDATE_ROUNDS = 3;
+    public static final int BTREE_MULTITHREAD_NUM_OPERATIONS = 200;
+    // Test params for LSMBTree only.
+    public static final int LSM_BTREE_BULKLOAD_ROUNDS = 5;
+    public static final int LSM_BTREE_MAX_TREES_TO_MERGE = 10;
+
+    // Mem configuration for RTree.
+    public static final int RTREE_PAGE_SIZE = 512;
+    public static final int RTREE_NUM_PAGES = 1000;
+    public static final int RTREE_MAX_OPEN_FILES = 10;
+    public static final int RTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for LSMRTree and LSMRTreeWithAntiMatterTuples.
+    public static final int LSM_RTREE_DISK_PAGE_SIZE = 512;
+    public static final int LSM_RTREE_DISK_NUM_PAGES = 1000;
+    public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = 2000;
+    public static final int LSM_RTREE_MEM_PAGE_SIZE = 512;
+    public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
+    public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for BTree.
+    public static final int BTREE_PAGE_SIZE = 256;
+    public static final int BTREE_NUM_PAGES = 100;
+    public static final int BTREE_MAX_OPEN_FILES = 10;
+    public static final int BTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for LSMBTree.
+    public static final int LSM_BTREE_DISK_PAGE_SIZE = 256;
+    public static final int LSM_BTREE_DISK_NUM_PAGES = 1000;
+    public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = 200;
+    public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
+    public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
+    public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
+
+    // Mem configuration for Inverted Index.
+    public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
+    public static final int LSM_INVINDEX_DISK_NUM_PAGES = 1000;
+    public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
+    public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
+    public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
+    public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+    // Test parameters.
+    public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 100;
+    // Used for full-fledged search test.
+    public static final int LSM_INVINDEX_NUM_DOC_QUERIES = 1000;
+    public static final int LSM_INVINDEX_NUM_RANDOM_QUERIES = 1000;
+    // Used for non-search tests to sanity check index searches.
+    public static final int LSM_INVINDEX_TINY_NUM_DOC_QUERIES = 200;
+    public static final int LSM_INVINDEX_TINY_NUM_RANDOM_QUERIES = 200;
+    public static final int LSM_INVINDEX_NUM_BULKLOAD_ROUNDS = 5;
+    public static final int LSM_INVINDEX_MAX_TREES_TO_MERGE = 5;
+    public static final int LSM_INVINDEX_NUM_INSERT_ROUNDS = 3;
+    public static final int LSM_INVINDEX_NUM_DELETE_ROUNDS = 3;
+    // Allocate a generous size to make sure we have enough elements for all tests.
+    public static final int LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE = 1000000;
+    public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 200;
+
+    // Test params for BloomFilter
+    public static final int BLOOM_FILTER_NUM_TUPLES_TO_INSERT = 100;
+
+    // Mem configuration for BloomFilter.
+    public static final int BLOOM_FILTER_PAGE_SIZE = 256;
+    public static final int BLOOM_FILTER_NUM_PAGES = 1000;
+    public static final int BLOOM_FILTER_MAX_OPEN_FILES = 10;
+    public static final int BLOOM_FILTER_HYRACKS_FRAME_SIZE = 128;
+
+}
+
+/* ORIGINAL TEST PARAMETERS: DO NOT EDIT!
+// Test params for RTree, LSMRTree and LSMRTreeWithAntiMatterTuples.
+public static final int RTREE_NUM_TUPLES_TO_INSERT = 10000;
+public static final int RTREE_NUM_INSERT_ROUNDS = 2;
+public static final int RTREE_NUM_DELETE_ROUNDS = 2;
+public static final int RTREE_MULTITHREAD_NUM_OPERATIONS = 10000;
+// Test params for LSMRTree and LSMRTreeWithAntiMatterTuples.
+public static final int LSM_RTREE_BULKLOAD_ROUNDS = 5;
+public static final int LSM_RTREE_MAX_TREES_TO_MERGE = 3;	
+
+// Test params for BTree, LSMBTree.
+public static final int BTREE_NUM_TUPLES_TO_INSERT = 10000;
+public static final int BTREE_NUM_INSERT_ROUNDS = 3;
+public static final int BTREE_NUM_DELETE_ROUNDS = 3;
+public static final int BTREE_NUM_UPDATE_ROUNDS = 3;
+public static final int BTREE_MULTITHREAD_NUM_OPERATIONS = 10000;
+// Test params for LSMBTree only.
+public static final int LSM_BTREE_BULKLOAD_ROUNDS = 5;
+public static final int LSM_BTREE_MAX_TREES_TO_MERGE = 10;
+	
+	
+// Mem configuration for RTree.
+public static final int RTREE_PAGE_SIZE = 512;
+public static final int RTREE_NUM_PAGES = 1000;
+public static final int RTREE_MAX_OPEN_FILES = 10;
+public static final int RTREE_HYRACKS_FRAME_SIZE = 128;
+	
+// Mem configuration for LSMRTree and LSMRTreeWithAntiMatterTuples.
+public static final int LSM_RTREE_DISK_PAGE_SIZE = 256;
+public static final int LSM_RTREE_DISK_NUM_PAGES = 1000;
+public static final int LSM_RTREE_DISK_MAX_OPEN_FILES = 2000;
+public static final int LSM_RTREE_MEM_PAGE_SIZE = 256;
+public static final int LSM_RTREE_MEM_NUM_PAGES = 1000;
+public static final int LSM_RTREE_HYRACKS_FRAME_SIZE = 128;
+	
+// Mem configuration for BTree.
+public static final int BTREE_PAGE_SIZE = 256;
+public static final int BTREE_NUM_PAGES = 100;
+public static final int BTREE_MAX_OPEN_FILES = 10;
+public static final int BTREE_HYRACKS_FRAME_SIZE = 128;
+	
+// Mem configuration for LSMBTree.
+public static final int LSM_BTREE_DISK_PAGE_SIZE = 256;
+public static final int LSM_BTREE_DISK_NUM_PAGES = 1000;
+public static final int LSM_BTREE_DISK_MAX_OPEN_FILES = 200;
+public static final int LSM_BTREE_MEM_PAGE_SIZE = 256;
+public static final int LSM_BTREE_MEM_NUM_PAGES = 100;
+public static final int LSM_BTREE_HYRACKS_FRAME_SIZE = 128;
+
+// Mem configuration for Inverted Index.
+public static final int INVINDEX_PAGE_SIZE = 32768;
+public static final int INVINDEX_NUM_PAGES = 100;
+public static final int INVINDEX_MAX_OPEN_FILES = 10;
+public static final int INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+
+// Mem configuration for Inverted Index.
+public static final int LSM_INVINDEX_DISK_PAGE_SIZE = 1024;
+public static final int LSM_INVINDEX_DISK_NUM_PAGES = 1000;
+public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
+public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
+public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
+public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
+// Test parameters.
+public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 10000;
+// Used for full-fledged search test.
+public static final int LSM_INVINDEX_NUM_DOC_QUERIES = 1000;
+public static final int LSM_INVINDEX_NUM_RANDOM_QUERIES = 1000;
+// Used for non-search tests to sanity check index searches.
+public static final int LSM_INVINDEX_TINY_NUM_DOC_QUERIES = 200;
+public static final int LSM_INVINDEX_TINY_NUM_RANDOM_QUERIES = 200;
+public static final int LSM_INVINDEX_NUM_BULKLOAD_ROUNDS = 5;
+public static final int LSM_INVINDEX_MAX_TREES_TO_MERGE = 5;
+public static final int LSM_INVINDEX_NUM_INSERT_ROUNDS = 3;
+public static final int LSM_INVINDEX_NUM_DELETE_ROUNDS = 3;
+// Allocate a generous size to make sure we have enough elements for all tests.
+public static final int LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE = 1000000;
+public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 10000;
+*/
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
index 198ac58..54ad1fe 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeBulkLoadTest.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeBulkLoadTest extends AbstractRTreeTestDriver {
@@ -27,15 +28,19 @@
     private final RTreeTestUtils rTreeTestUtils;
     private final int bulkLoadRounds;
 
-    public AbstractRTreeBulkLoadTest(int bulkLoadRounds) {
+    public AbstractRTreeBulkLoadTest(int bulkLoadRounds, boolean testRstarPolicy) {
+        super(testRstarPolicy);
         this.bulkLoadRounds = bulkLoadRounds;
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception {
-        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         for (int i = 0; i < bulkLoadRounds; i++) {
             // We assume all fieldSerdes are of the same type. Check the first
             // one to determine which field types to generate.
@@ -49,7 +54,8 @@
             rTreeTestUtils.checkDiskOrderScan(ctx);
             rTreeTestUtils.checkRangeSearch(ctx, key);
         }
-        ctx.getIndex().close();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java
index e70f433..18d042b 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeDeleteTest.java
@@ -20,23 +20,29 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeDeleteTest extends AbstractRTreeTestDriver {
 
     private final RTreeTestUtils rTreeTestUtils;
 
-    private static final int numInsertRounds = 2;
-    private static final int numDeleteRounds = 2;
+    private static final int numInsertRounds = AccessMethodTestsConfig.RTREE_NUM_INSERT_ROUNDS;
+    private static final int numDeleteRounds = AccessMethodTestsConfig.RTREE_NUM_DELETE_ROUNDS;
 
-    public AbstractRTreeDeleteTest() {
+    public AbstractRTreeDeleteTest(boolean testRstarPolicy) {
+    	super(testRstarPolicy);
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception {
-        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         for (int i = 0; i < numInsertRounds; i++) {
             // We assume all fieldSerdes are of the same type. Check the first
             // one to determine which field types to generate.
@@ -54,7 +60,8 @@
                 rTreeTestUtils.checkRangeSearch(ctx, key);
             }
         }
-        ctx.getIndex().close();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 7192c53..f93e9b6 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -27,21 +27,26 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.common.TestOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
@@ -52,16 +57,14 @@
 
     protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories) throws TreeIndexException;
-
-    protected abstract int getIndexFileId();
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType)
+            throws TreeIndexException;
 
     /**
-     * Two Dimensions Example.
-     * 
-     * Create an RTree index of two dimensions, where they keys are of type
-     * integer, and the payload is two integer values. Fill index with random
-     * values using insertions (not bulk load). Perform scans and range search.
+     * Two Dimensions Example. Create an RTree index of two dimensions, where
+     * they keys are of type integer, and the payload is two integer values.
+     * Fill index with random values using insertions (not bulk load). Perform
+     * scans and range search.
      */
     @Test
     public void twoDimensionsExample() throws Exception {
@@ -106,10 +109,10 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -117,7 +120,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
             int p1x = rnd.nextInt();
@@ -156,15 +160,245 @@
 
         rangeSearch(rtreeCmpFactories, indexAccessor, fieldSerdes, key);
 
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Two Dimensions Example.
-     * 
-     * Create an RTree index of three dimensions, where they keys are of type
-     * double, and the payload is one double value. Fill index with random
-     * values using insertions (not bulk load). Perform scans and range search.
+     * This test the rtree page split. Originally this test didn't pass since
+     * the rtree assumes always that there will be enough space for the new
+     * tuple after split. Now it passes since if there is not space in the
+     * designated page, then we will just insert it in the other split page.
+     */
+    @Test
+    public void rTreePageSplitTestExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree page split test.");
+        }
+
+        // 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] = UTF8StringPointable.TYPE_TRAITS;
+        // Declare field serdes.
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+
+        // Declare RTree keys.
+        int rtreeKeyFieldCount = 4;
+        IBinaryComparatorFactory[] rtreeCmpFactories = new IBinaryComparatorFactory[rtreeKeyFieldCount];
+        rtreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        // Declare BTree keys, this will only be used for LSMRTree
+        int btreeKeyFieldCount = 5;
+        IBinaryComparatorFactory[] btreeCmpFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+        btreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[4] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                rtreeCmpFactories.length, IntegerPointable.FACTORY);
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+
+        treeIndex.create();
+        treeIndex.activate();
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+
+        int p1x = rnd.nextInt();
+        int p1y = rnd.nextInt();
+        int p2x = rnd.nextInt();
+        int p2y = rnd.nextInt();
+        String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        treeIndex.deactivate();
+        treeIndex.destroy();
+    }
+
+    /**
+     * This test the r*tree page split. Originally this test didn't pass since
+     * the r*tree assumes always that there will be enough space for the new
+     * tuple after split. Now it passes since if there is not space in the
+     * designated page, then we will just insert it in the other split page.
+     */
+    @Test
+    public void rStarTreePageSplitTestExample() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("R*Tree page split test.");
+        }
+
+        // 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] = UTF8StringPointable.TYPE_TRAITS;
+        // Declare field serdes.
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+
+        // Declare RTree keys.
+        int rtreeKeyFieldCount = 4;
+        IBinaryComparatorFactory[] rtreeCmpFactories = new IBinaryComparatorFactory[rtreeKeyFieldCount];
+        rtreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        rtreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+
+        // Declare BTree keys, this will only be used for LSMRTree
+        int btreeKeyFieldCount = 5;
+        IBinaryComparatorFactory[] btreeCmpFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+        btreeCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[1] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[2] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[3] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
+        btreeCmpFactories[4] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
+
+        // create value providers
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                rtreeCmpFactories.length, IntegerPointable.FACTORY);
+
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RSTARTREE);
+
+        treeIndex.create();
+        treeIndex.activate();
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+        ArrayTupleReference tuple = new ArrayTupleReference();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(TestOperationCallback.INSTANCE,
+                TestOperationCallback.INSTANCE);
+
+        int p1x = rnd.nextInt();
+        int p1y = rnd.nextInt();
+        int p2x = rnd.nextInt();
+        int p2y = rnd.nextInt();
+        String data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        p1x = rnd.nextInt();
+        p1y = rnd.nextInt();
+        p2x = rnd.nextInt();
+        p2y = rnd.nextInt();
+        data = "XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX";
+        TupleUtils.createTuple(tb, tuple, fieldSerdes, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
+                Math.max(p1y, p2y), data);
+        indexAccessor.insert(tuple);
+
+        treeIndex.deactivate();
+        treeIndex.destroy();
+    }
+
+    /**
+     * Two Dimensions Example. Create an RTree index of three dimensions, where
+     * they keys are of type double, and the payload is one double value. Fill
+     * index with random values using insertions (not bulk load). Perform scans
+     * and range search.
      */
     @Test
     public void threeDimensionsExample() throws Exception {
@@ -213,10 +447,10 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, DoublePointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         long start = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -224,7 +458,8 @@
         }
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
         int numInserts = 10000;
         for (int i = 0; i < numInserts; i++) {
             double p1x = rnd.nextDouble();
@@ -265,16 +500,15 @@
 
         rangeSearch(rtreeCmpFactories, indexAccessor, fieldSerdes, key);
 
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Deletion Example.
-     * 
-     * Create an RTree index of two dimensions, where they keys are of type
-     * integer, and the payload is one integer value. Fill index with random
-     * values using insertions, then delete entries one-by-one. Repeat procedure
-     * a few times on same RTree.
+     * Deletion Example. Create an RTree index of two dimensions, where they
+     * keys are of type integer, and the payload is one integer value. Fill
+     * index with random values using insertions, then delete entries
+     * one-by-one. Repeat procedure a few times on same RTree.
      */
     @Test
     public void deleteExample() throws Exception {
@@ -312,14 +546,15 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
 
         int runs = 3;
         for (int run = 0; run < runs; run++) {
@@ -395,14 +630,12 @@
                 break;
             }
         }
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     /**
-     * Bulk load example.
-     * 
-     * Load a tree with 10,000 tuples.
-     * 
+     * Bulk load example. Load a tree with 10,000 tuples.
      */
     @Test
     public void bulkLoadExample() throws Exception {
@@ -444,10 +677,10 @@
         IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
                 rtreeCmpFactories.length, IntegerPointable.FACTORY);
 
-        int indexFileId = getIndexFileId();
-        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        treeIndex.create(indexFileId);
-        treeIndex.open(indexFileId);
+        ITreeIndex treeIndex = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                valueProviderFactories, RTreePolicyType.RTREE);
+        treeIndex.create();
+        treeIndex.activate();
 
         // Load records.
         int numInserts = 10000;
@@ -455,7 +688,7 @@
             LOGGER.info("Bulk loading " + numInserts + " tuples");
         }
         long start = System.currentTimeMillis();
-        IIndexBulkLoadContext bulkLoadCtx = treeIndex.beginBulkLoad(0.7f);
+        IIndexBulkLoader bulkLoader = treeIndex.createBulkLoader(0.7f, false, numInserts);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
@@ -469,16 +702,17 @@
 
             TupleUtils.createIntegerTuple(tb, tuple, Math.min(p1x, p2x), Math.min(p1y, p2y), Math.max(p1x, p2x),
                     Math.max(p1y, p2y), pk);
-            treeIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
+            bulkLoader.add(tuple);
         }
 
-        treeIndex.endBulkLoad(bulkLoadCtx);
+        bulkLoader.end();
         long end = System.currentTimeMillis();
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(numInserts + " tuples loaded in " + (end - start) + "ms");
         }
 
-        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor();
+        IIndexAccessor indexAccessor = (IIndexAccessor) treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
 
         // Build key.
         ArrayTupleBuilder keyTb = new ArrayTupleBuilder(rtreeKeyFieldCount);
@@ -487,7 +721,8 @@
 
         rangeSearch(rtreeCmpFactories, indexAccessor, fieldSerdes, key);
 
-        treeIndex.close();
+        treeIndex.deactivate();
+        treeIndex.destroy();
     }
 
     private void scan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
@@ -511,14 +746,13 @@
         }
     }
 
-    private void diskOrderScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes)
-            throws Exception {
+    private void diskOrderScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes) throws Exception {
         try {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Disk-Order Scan:");
             }
             ITreeIndexAccessor treeIndexAccessor = (ITreeIndexAccessor) indexAccessor;
-            TreeDiskOrderScanCursor diskOrderCursor = (TreeDiskOrderScanCursor) treeIndexAccessor
+            TreeIndexDiskOrderScanCursor diskOrderCursor = (TreeIndexDiskOrderScanCursor) treeIndexAccessor
                     .createDiskOrderScanCursor();
             treeIndexAccessor.diskOrderScan(diskOrderCursor);
             try {
@@ -540,12 +774,12 @@
                 LOGGER.info("Ignoring disk-order scan since it's not supported.");
             }
         } catch (ClassCastException e) {
-			// Ignore exception because IIndexAccessor sometimes isn't
-			// an ITreeIndexAccessor, e.g., for the LSMRTree.
-			if (LOGGER.isLoggable(Level.INFO)) {
-				LOGGER.info("Ignoring disk-order scan since it's not supported.");
-			}
-		}
+            // Ignore exception because IIndexAccessor sometimes isn't
+            // an ITreeIndexAccessor, e.g., for the LSMRTree.
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring disk-order scan since it's not supported.");
+            }
+        }
     }
 
     private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor indexAccessor,
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java
index cdd6ee0..eb90989 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeInsertTest.java
@@ -20,29 +20,32 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 
 /**
  * Tests the RTree insert operation with integer and double fields using various
  * numbers of dimensions and payload fields.
- * 
  * Each tests first fills an RTree with randomly generated tuples. We compare
  * the following operations against expected results: 1. RTree scan. 3.
  * Disk-order scan. 4. Range search.
- * 
  */
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeInsertTest extends AbstractRTreeTestDriver {
 
     private final RTreeTestUtils rTreeTestUtils;
 
-    public AbstractRTreeInsertTest() {
+    public AbstractRTreeInsertTest(boolean testRstarPolicy) {
+    	super(testRstarPolicy);
         this.rTreeTestUtils = new RTreeTestUtils();
     }
 
     @Override
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception {
-        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys);
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception {
+        AbstractRTreeTestContext ctx = createTestContext(fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType);
+        ctx.getIndex().create();
+        ctx.getIndex().activate();
         // We assume all fieldSerdes are of the same type. Check the first one
         // to determine which field types to generate.
         if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) {
@@ -54,7 +57,8 @@
         rTreeTestUtils.checkScan(ctx);
         rTreeTestUtils.checkDiskOrderScan(ctx);
         rTreeTestUtils.checkRangeSearch(ctx, key);
-        ctx.getIndex().close();
+        ctx.getIndex().deactivate();
+        ctx.getIndex().destroy();
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
index 2c185a5f..0c1b9a5 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeMultiThreadTest.java
@@ -31,25 +31,32 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestWorkerFactory;
-import edu.uci.ics.hyracks.storage.am.common.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.IndexMultiThreadTestDriver;
 import edu.uci.ics.hyracks.storage.am.common.TestWorkloadConf;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexMultiThreadTestDriver;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeMultiThreadTest {
 
+    protected final boolean testRstarPolicy;
+
+    public AbstractRTreeMultiThreadTest(boolean testRstarPolicy) {
+        this.testRstarPolicy = testRstarPolicy;
+    }
+
     protected final Logger LOGGER = Logger.getLogger(AbstractRTreeMultiThreadTest.class.getName());
 
     // Machine-specific number of threads to use for testing.
     protected final int REGULAR_NUM_THREADS = Runtime.getRuntime().availableProcessors();
     // Excessive number of threads for testing.
     protected final int EXCESSIVE_NUM_THREADS = Runtime.getRuntime().availableProcessors() * 4;
-    protected final int NUM_OPERATIONS = 5000;
+    protected final int NUM_OPERATIONS = AccessMethodTestsConfig.RTREE_MULTITHREAD_NUM_OPERATIONS;
 
     protected ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
 
@@ -59,27 +66,19 @@
 
     protected abstract ITreeIndex createTreeIndex(ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
-            IPrimitiveValueProviderFactory[] valueProviderFactories) throws TreeIndexException;
+            IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType)
+            throws TreeIndexException;
 
-    protected abstract int getFileId();
-
-    protected abstract ITreeIndexTestWorkerFactory getWorkerFactory();
+    protected abstract IIndexTestWorkerFactory getWorkerFactory();
 
     protected abstract ArrayList<TestWorkloadConf> getTestWorkloadConf();
 
     protected abstract String getIndexTypeName();
 
-    protected static float[] getUniformOpProbs(TestOperation[] ops) {
-        float[] opProbs = new float[ops.length];
-        for (int i = 0; i < ops.length; i++) {
-            opProbs[i] = 1.0f / (float) ops.length;
-        }
-        return opProbs;
-    }
-
     protected void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, int numThreads,
-            TestWorkloadConf conf, String dataMsg) throws HyracksException, InterruptedException, TreeIndexException {
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType,
+            int numThreads, TestWorkloadConf conf, String dataMsg) throws HyracksException, InterruptedException,
+            TreeIndexException {
         setUp();
 
         if (LOGGER.isLoggable(Level.INFO)) {
@@ -93,15 +92,16 @@
         IBinaryComparatorFactory[] btreeCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
                 fieldSerdes.length);
 
-        ITreeIndex index = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories);
-        ITreeIndexTestWorkerFactory workerFactory = getWorkerFactory();
+        ITreeIndex index = createTreeIndex(typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
+                rtreePolicyType);
+        IIndexTestWorkerFactory workerFactory = getWorkerFactory();
 
         // 4 batches per thread.
         int batchSize = (NUM_OPERATIONS / numThreads) / 4;
 
-        TreeIndexMultiThreadTestDriver driver = new TreeIndexMultiThreadTestDriver(index, workerFactory, fieldSerdes,
-                conf.ops, conf.opProbs);
-        driver.init(getFileId());
+        IndexMultiThreadTestDriver driver = new IndexMultiThreadTestDriver(index, workerFactory, fieldSerdes, conf.ops,
+                conf.opProbs);
+        driver.init();
         long[] times = driver.run(numThreads, 1, NUM_OPERATIONS, batchSize);
         driver.deinit();
 
@@ -113,7 +113,7 @@
     }
 
     @Test
-    public void twoDimensionsInt() throws InterruptedException, HyracksException, TreeIndexException {
+    public void rtreeTwoDimensionsInt() throws InterruptedException, HyracksException, TreeIndexException {
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -125,13 +125,36 @@
         String dataMsg = "Two Dimensions Of Integer Values";
 
         for (TestWorkloadConf conf : workloadConfs) {
-            runTest(fieldSerdes, valueProviderFactories, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
-            runTest(fieldSerdes, valueProviderFactories, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, EXCESSIVE_NUM_THREADS, conf,
+                    dataMsg);
         }
     }
 
     @Test
-    public void fourDimensionsDouble() throws InterruptedException, HyracksException, TreeIndexException {
+    public void rtreeTwoDimensionsDouble() throws Exception {
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+
+        String dataMsg = "Two Dimensions Of Double Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, EXCESSIVE_NUM_THREADS, conf,
+                    dataMsg);
+        }
+
+    }
+
+    @Test
+    public void rtreeFourDimensionsDouble() throws InterruptedException, HyracksException, TreeIndexException {
         ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
                 DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
                 DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
@@ -145,8 +168,95 @@
         String dataMsg = "Four Dimensions Of Double Values";
 
         for (TestWorkloadConf conf : workloadConfs) {
-            runTest(fieldSerdes, valueProviderFactories, numKeys, REGULAR_NUM_THREADS, conf, dataMsg);
-            runTest(fieldSerdes, valueProviderFactories, numKeys, EXCESSIVE_NUM_THREADS, conf, dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, EXCESSIVE_NUM_THREADS, conf,
+                    dataMsg);
         }
     }
+
+    @Test
+    public void rstartreeTwoDimensionsInt() throws InterruptedException, HyracksException, TreeIndexException {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree Multithread Test With Two Dimensions With Integer Keys.");
+            }
+            return;
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, IntegerPointable.FACTORY);
+
+        String dataMsg = "Two Dimensions Of Integer Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, EXCESSIVE_NUM_THREADS,
+                    conf, dataMsg);
+        }
+    }
+
+    @Test
+    public void rstartreeTwoDimensionsDouble() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree Multithread Test With Two Dimensions With Double Keys.");
+            }
+            return;
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+
+        String dataMsg = "Two Dimensions Of Double Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, EXCESSIVE_NUM_THREADS,
+                    conf, dataMsg);
+        }
+
+    }
+
+    @Test
+    public void rstartreeFourDimensionsDouble() throws InterruptedException, HyracksException, TreeIndexException {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree Multithread Test With Four Dimensions With Double Keys.");
+            }
+            return;
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 8;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+
+        String dataMsg = "Four Dimensions Of Double Values";
+
+        for (TestWorkloadConf conf : workloadConfs) {
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, REGULAR_NUM_THREADS, conf,
+                    dataMsg);
+            runTest(fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RSTARTREE, EXCESSIVE_NUM_THREADS,
+                    conf, dataMsg);
+        }
+    }
+
 }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
index 9affc47..eed8df6 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestContext.java
@@ -15,17 +15,17 @@
 
 package edu.uci.ics.hyracks.storage.am.rtree;
 
-import java.util.ArrayList;
 import java.util.Collection;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.IndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.util.HashMultiSet;
 
 @SuppressWarnings("rawtypes")
-public abstract class AbstractRTreeTestContext extends TreeIndexTestContext<RTreeCheckTuple> {
-    private final ArrayList<RTreeCheckTuple> checkTuples = new ArrayList<RTreeCheckTuple>();
-
+public abstract class AbstractRTreeTestContext extends IndexTestContext<RTreeCheckTuple> {
+    private final HashMultiSet<RTreeCheckTuple> checkTuples = new HashMultiSet<RTreeCheckTuple>();
+	
     public AbstractRTreeTestContext(ISerializerDeserializer[] fieldSerdes, ITreeIndex treeIndex) {
         super(fieldSerdes, treeIndex);
     }
@@ -34,5 +34,4 @@
     public Collection<RTreeCheckTuple> getCheckTuples() {
         return checkTuples;
     }
-
 }
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
index 10f4364..de92823 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTestDriver.java
@@ -29,26 +29,36 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
 @SuppressWarnings("rawtypes")
 public abstract class AbstractRTreeTestDriver {
+    protected final boolean testRstarPolicy;
+
+    public AbstractRTreeTestDriver(boolean testRstarPolicy) {
+        this.testRstarPolicy = testRstarPolicy;
+    }
+
     protected final Logger LOGGER = Logger.getLogger(AbstractRTreeTestDriver.class.getName());
 
-    protected static final int numTuplesToInsert = 5000;
+    protected static final int numTuplesToInsert = AccessMethodTestsConfig.RTREE_NUM_TUPLES_TO_INSERT;
 
     protected abstract AbstractRTreeTestContext createTestContext(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys) throws Exception;
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, RTreePolicyType rtreePolicyType)
+            throws Exception;
 
     protected abstract Random getRandom();
 
     protected abstract void runTest(ISerializerDeserializer[] fieldSerdes,
-            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key) throws Exception;
+            IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeys, ITupleReference key,
+            RTreePolicyType rtreePolicyType) throws Exception;
 
     protected abstract String getTestOpName();
 
     @Test
-    public void twoDimensionsInt() throws Exception {
+    public void rtreeTwoDimensionsInt() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Integer Keys.");
         }
@@ -64,12 +74,12 @@
         // and the top right coordinates are 1000, 1000
         ITupleReference key = TupleUtils.createIntegerTuple(-1000, -1000, 1000, 1000);
 
-        runTest(fieldSerdes, valueProviderFactories, numKeys, key);
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RTREE);
 
     }
 
     @Test
-    public void twoDimensionsDouble() throws Exception {
+    public void rtreeTwoDimensionsDouble() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Double Keys.");
         }
@@ -85,12 +95,12 @@
         // -1000.0 and the top right coordinates are 1000.0, 1000.0
         ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, 1000.0, 1000.0);
 
-        runTest(fieldSerdes, valueProviderFactories, numKeys, key);
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RTREE);
 
     }
 
     @Test
-    public void fourDimensionsDouble() throws Exception {
+    public void rtreeFourDimensionsDouble() throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("RTree " + getTestOpName() + " Test With Four Dimensions With Double Keys.");
         }
@@ -110,7 +120,90 @@
         ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, -1000.0, -1000.0, 1000.0, 1000.0, 1000.0,
                 1000.0);
 
-        runTest(fieldSerdes, valueProviderFactories, numKeys, key);
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RTREE);
+    }
 
+    @Test
+    public void rstartreeTwoDimensionsInt() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree " + getTestOpName() + " Test With Two Dimensions With Integer Keys.");
+            }
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Integer Keys.");
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, IntegerPointable.FACTORY);
+        // Range search, the rectangle bottom left coordinates are -1000, -1000
+        // and the top right coordinates are 1000, 1000
+        ITupleReference key = TupleUtils.createIntegerTuple(-1000, -1000, 1000, 1000);
+
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RSTARTREE);
+
+    }
+
+    @Test
+    public void rstartreeTwoDimensionsDouble() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree " + getTestOpName() + " Test With Two Dimensions With Double Keys.");
+            }
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree " + getTestOpName() + " Test With Two Dimensions With Double Keys.");
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 4;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+        // Range search, the rectangle bottom left coordinates are -1000.0,
+        // -1000.0 and the top right coordinates are 1000.0, 1000.0
+        ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, 1000.0, 1000.0);
+
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RSTARTREE);
+
+    }
+
+    @Test
+    public void rstartreeFourDimensionsDouble() throws Exception {
+        if (!testRstarPolicy) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("Ignoring RTree " + getTestOpName() + " Test With Four Dimensions With Double Keys.");
+            }
+            return;
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RTree " + getTestOpName() + " Test With Four Dimensions With Double Keys.");
+        }
+
+        ISerializerDeserializer[] fieldSerdes = { DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+                DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE };
+
+        int numKeys = 8;
+        IPrimitiveValueProviderFactory[] valueProviderFactories = RTreeUtils.createPrimitiveValueProviderFactories(
+                numKeys, DoublePointable.FACTORY);
+        // Range search, the rectangle bottom left coordinates are -1000.0,
+        // -1000.0, -1000.0, -1000.0 and the top right coordinates are 1000.0,
+        // 1000.0, 1000.0, 1000.0
+        ITupleReference key = TupleUtils.createDoubleTuple(-1000.0, -1000.0, -1000.0, -1000.0, 1000.0, 1000.0, 1000.0,
+                1000.0);
+
+        runTest(fieldSerdes, valueProviderFactories, numKeys, key, RTreePolicyType.RSTARTREE);
     }
 }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java
index 98800e5..c498136 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeCheckTuple.java
@@ -27,8 +27,8 @@
     @Override
     public boolean equals(Object o) {
         RTreeCheckTuple<T> other = (RTreeCheckTuple<T>) o;
-        for (int i = 0; i < tuple.length; i++) {
-            int cmp = tuple[i].compareTo(other.get(i));
+        for (int i = 0; i < fields.length; i++) {
+            int cmp = fields[i].compareTo(other.getField(i));
             if (cmp != 0) {
                 return false;
             }
@@ -41,11 +41,11 @@
         int maxFieldPos = numKeys / 2;
         for (int i = 0; i < maxFieldPos; i++) {
             int j = maxFieldPos + i;
-            int cmp = tuple[i].compareTo(other.get(j));
+            int cmp = fields[i].compareTo(other.getField(j));
             if (cmp > 0) {
                 return false;
             }
-            cmp = tuple[j].compareTo(other.get(i));
+            cmp = fields[j].compareTo(other.getField(i));
             if (cmp < 0) {
                 return false;
             }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
index a23f375..067c6cb 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -15,12 +15,13 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
-import edu.uci.ics.hyracks.storage.am.common.ITreeIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.common.IIndexTestContext;
 import edu.uci.ics.hyracks.storage.am.common.TreeIndexTestUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.HashMultiSet;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 
@@ -32,9 +33,9 @@
 
     @SuppressWarnings("unchecked")
     // Create a new ArrayList containing the elements satisfying the search key
-    public ArrayList<RTreeCheckTuple> getRangeSearchExpectedResults(ArrayList<RTreeCheckTuple> checkTuples,
+    public HashMultiSet<RTreeCheckTuple> getRangeSearchExpectedResults(Collection<RTreeCheckTuple> checkTuples,
             RTreeCheckTuple key) {
-        ArrayList<RTreeCheckTuple> expectedResult = new ArrayList<RTreeCheckTuple>();
+        HashMultiSet<RTreeCheckTuple> expectedResult = new HashMultiSet<RTreeCheckTuple>();
         Iterator<RTreeCheckTuple> iter = checkTuples.iterator();
         while (iter.hasNext()) {
             RTreeCheckTuple t = iter.next();
@@ -45,7 +46,7 @@
         return expectedResult;
     }
 
-    public void checkRangeSearch(ITreeIndexTestContext ictx, ITupleReference key) throws Exception {
+    public void checkRangeSearch(IIndexTestContext ictx, ITupleReference key) throws Exception {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Testing Range Search.");
         }
@@ -61,14 +62,14 @@
         RTreeCheckTuple keyCheck = (RTreeCheckTuple) createCheckTupleFromTuple(key, ctx.getFieldSerdes(),
                 cmp.getKeyFieldCount());
 
-        ArrayList<RTreeCheckTuple> expectedResult = null;
+        HashMultiSet<RTreeCheckTuple> expectedResult = null;
 
-        expectedResult = getRangeSearchExpectedResults((ArrayList<RTreeCheckTuple>) ctx.getCheckTuples(), keyCheck);
+        expectedResult = getRangeSearchExpectedResults(ctx.getCheckTuples(), keyCheck);
         checkExpectedResults(searchCursor, expectedResult, ctx.getFieldSerdes(), ctx.getKeyFieldCount(), null);
     }
 
     @SuppressWarnings("unchecked")
-    public void insertDoubleTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void insertDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         double[] fieldValues = new double[ctx.getFieldCount()];
@@ -111,7 +112,7 @@
             fieldValues[k] = secondValue;
         }
     }
-    
+
     private void setDoublePayloadFields(double[] fieldValues, int numKeyFields, int numFields) {
         for (int j = numKeyFields; j < numFields; j++) {
             fieldValues[j] = doublePayloadValue++;
@@ -122,13 +123,13 @@
     protected CheckTuple createDoubleCheckTuple(double[] fieldValues, int numKeyFields) {
         RTreeCheckTuple<Double> checkTuple = new RTreeCheckTuple<Double>(fieldValues.length, numKeyFields);
         for (double v : fieldValues) {
-            checkTuple.add(v);
+            checkTuple.appendField(v);
         }
         return checkTuple;
     }
 
     @SuppressWarnings("unchecked")
-    public void bulkLoadDoubleTuples(ITreeIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
+    public void bulkLoadDoubleTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception {
         int fieldCount = ctx.getFieldCount();
         int numKeyFields = ctx.getKeyFieldCount();
         double[] fieldValues = new double[ctx.getFieldCount()];
@@ -194,7 +195,7 @@
     protected CheckTuple createIntCheckTuple(int[] fieldValues, int numKeyFields) {
         RTreeCheckTuple<Integer> checkTuple = new RTreeCheckTuple<Integer>(fieldValues.length, numKeyFields);
         for (int v : fieldValues) {
-            checkTuple.add(v);
+            checkTuple.appendField(v);
         }
         return checkTuple;
     }
@@ -213,7 +214,7 @@
             fieldValues[k] = secondValue;
         }
     }
-    
+
     @Override
     protected void setIntPayloadFields(int[] fieldValues, int numKeyFields, int numFields) {
         for (int j = numKeyFields; j < numFields; j++) {
@@ -227,12 +228,12 @@
     }
 
     @Override
-    protected ArrayTupleBuilder createDeleteTupleBuilder(ITreeIndexTestContext ctx) {
+    protected ArrayTupleBuilder createDeleteTupleBuilder(IIndexTestContext ctx) {
         return new ArrayTupleBuilder(ctx.getFieldCount());
     }
 
     @Override
-    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, ITreeIndexTestContext ctx)
+    protected boolean checkDiskOrderScanResult(ITupleReference tuple, CheckTuple checkTuple, IIndexTestContext ctx)
             throws HyracksDataException {
         return ctx.getCheckTuples().contains(checkTuple);
     }
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexLifecycleManagerProvider.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexLifecycleManagerProvider.java
new file mode 100644
index 0000000..3a4d3e7
--- /dev/null
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexLifecycleManagerProvider.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hyracks.test.support;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+
+public class TestIndexLifecycleManagerProvider implements IIndexLifecycleManagerProvider {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+        return TestStorageManagerComponentHolder.getIndexLifecycleManager(ctx);
+    }
+
+}
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
deleted file mode 100644
index 27d50f5..0000000
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- * 
- *     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 edu.uci.ics.hyracks.test.support;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-
-public class TestIndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
-        return TestStorageManagerComponentHolder.getIndexRegistry(ctx);
-    }
-}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
index 0bd872c..0c64d7e 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestNCApplicationContext.java
@@ -38,11 +38,6 @@
     }
 
     @Override
-    public ClassLoader getClassLoader() {
-        return getClass().getClassLoader();
-    }
-
-    @Override
     public Serializable getDistributedState() {
         return distributedState;
     }
@@ -63,15 +58,9 @@
     }
 
     @Override
-    public String getApplicationName() {
-        // TODO Auto-generated method stub
-        return null;
-    }
-
-    @Override
     public void setMessageBroker(IMessageBroker staticticsConnector) {
         // TODO Auto-generated method stub
-        
+
     }
 
     @Override
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index ce5e989..fdb2100 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -23,8 +23,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.DelayPageCleanerPolicy;
@@ -34,13 +34,20 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceRepositoryFactory;
 
 public class TestStorageManagerComponentHolder {
     private static IBufferCache bufferCache;
     private static IFileMapProvider fileMapProvider;
-    private static IndexRegistry<IIndex> indexRegistry;
     private static IOManager ioManager;
+    private static ILocalResourceRepository localResourceRepository;
+    private static IIndexLifecycleManager lcManager;
+    private static ResourceIdFactory resourceIdFactory;
 
     private static int pageSize;
     private static int numPages;
@@ -52,7 +59,15 @@
         TestStorageManagerComponentHolder.maxOpenFiles = maxOpenFiles;
         bufferCache = null;
         fileMapProvider = null;
-        indexRegistry = null;
+        localResourceRepository = null;
+        lcManager = null;
+    }
+
+    public synchronized static IIndexLifecycleManager getIndexLifecycleManager(IHyracksTaskContext ctx) {
+        if (lcManager == null) {
+            lcManager = new IndexLifecycleManager();
+        }
+        return lcManager;
     }
 
     public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
@@ -73,13 +88,6 @@
         return fileMapProvider;
     }
 
-    public synchronized static IndexRegistry<IIndex> getIndexRegistry(IHyracksTaskContext ctx) {
-        if (indexRegistry == null) {
-            indexRegistry = new IndexRegistry<IIndex>();
-        }
-        return indexRegistry;
-    }
-
     public synchronized static IOManager getIOManager() throws HyracksException {
         if (ioManager == null) {
             List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
@@ -88,4 +96,31 @@
         }
         return ioManager;
     }
+
+    public synchronized static ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        if (localResourceRepository == null) {
+            try {
+                ILocalResourceRepositoryFactory localResourceRepositoryFactory = new TransientLocalResourceRepositoryFactory();
+                localResourceRepository = localResourceRepositoryFactory.createRepository();
+            } catch (HyracksException e) {
+                //In order not to change the IStorageManagerInterface due to the test code, throw runtime exception.
+                throw new IllegalArgumentException();
+            }
+        }
+        return localResourceRepository;
+    }
+
+    public synchronized static ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+        if (resourceIdFactory == null) {
+            try {
+                ResourceIdFactoryProvider resourceIdFactoryFactory = new ResourceIdFactoryProvider(
+                        getLocalResourceRepository(ctx));
+                resourceIdFactory = resourceIdFactoryFactory.createResourceIdFactory();
+            } catch (HyracksException e) {
+                //In order not to change the IStorageManagerInterface due to the test code, throw runtime exception.
+                throw new IllegalArgumentException();
+            }
+        }
+        return resourceIdFactory;
+    }
 }
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
index 4059ef0..26c7861 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
@@ -15,9 +15,12 @@
 package edu.uci.ics.hyracks.test.support;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
 public class TestStorageManagerInterface implements IStorageManagerInterface {
     private static final long serialVersionUID = 1L;
@@ -31,4 +34,14 @@
     public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
         return TestStorageManagerComponentHolder.getFileMapProvider(ctx);
     }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        return TestStorageManagerComponentHolder.getLocalResourceRepository(ctx);
+    }
+
+	@Override
+	public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+		return TestStorageManagerComponentHolder.getResourceIdFactory(ctx);
+	}
 }
\ No newline at end of file