Refactored BTree ops to share them with the RTree. All ops except the search op are currently sharable. The search op needs more thinking.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_indexes@477 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index b8d5f21..5dd5650 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -41,17 +41,18 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -128,7 +129,7 @@
         // run data generator on first nodecontroller given
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dataGen, splitNCs[0]);
 
-        IIndexRegistryProvider<BTree> btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
+        IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // prepare insertion into primary index
@@ -142,8 +143,8 @@
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
-        IBTreeInteriorFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
-        IBTreeLeafFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
+        ITreeIndexFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
+        ITreeIndexFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
 
         // the B-Tree expects its keyfields to be at the front of its input
         // tuple
@@ -155,11 +156,13 @@
         primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
         IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
 
+        ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+        
         // create operator descriptor
-        BTreeInsertUpdateDeleteOperatorDescriptor primaryInsert = new BTreeInsertUpdateDeleteOperatorDescriptor(spec,
-                recDesc, storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
+        TreeIndexInsertUpdateDeleteOperatorDescriptor primaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(spec,
+                recDesc, storageManager, treeIndexRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
                 primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, primaryFieldPermutation,
-                TreeIndexOp.TI_INSERT);
+                IndexOp.INSERT, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, primaryInsert, splitNCs);
 
         // prepare insertion into secondary index
@@ -171,9 +174,9 @@
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
-        IBTreeInteriorFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
+        ITreeIndexFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
                 secondaryTupleWriterFactory);
-        IBTreeLeafFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(secondaryTupleWriterFactory);
+        ITreeIndexFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(secondaryTupleWriterFactory);
 
         // the B-Tree expects its keyfields to be at the front of its input
         // tuple
@@ -185,10 +188,10 @@
         IFileSplitProvider secondarySplitProvider = JobHelper.createFileSplitProvider(splitNCs,
                 options.secondaryBTreeName);
         // create operator descriptor
-        BTreeInsertUpdateDeleteOperatorDescriptor secondaryInsert = new BTreeInsertUpdateDeleteOperatorDescriptor(spec,
-                recDesc, storageManager, btreeRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
+        TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(spec,
+                recDesc, storageManager, treeIndexRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
                 secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories,
-                secondaryFieldPermutation, TreeIndexOp.TI_INSERT);
+                secondaryFieldPermutation, IndexOp.INSERT, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, secondaryInsert, splitNCs);
 
         // end the insert pipeline at this sink operator
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 8fec2af..7e88bc1 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -39,16 +39,17 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -145,20 +146,21 @@
 
         // create factories and providers for B-Tree
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
-        IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
-        IIndexRegistryProvider<BTree> btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
+        ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
+        IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // the B-Tree expects its keyfields to be at the front of its input
         // tuple
         int[] fieldPermutation = { 2, 1, 3, 4 }; // map field 2 of input tuple
                                                  // to field 0 of B-Tree tuple,
-                                                 // etc.
+                                                 // etc.                
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
-        BTreeBulkLoadOperatorDescriptor btreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(spec, storageManager,
-                btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
-                comparatorFactories, fieldPermutation, 0.7f);
+        ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+        TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec, storageManager,
+                treeIndexRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
+                comparatorFactories, fieldPermutation, 0.7f, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // distribute the records from the datagen via hashing to the bulk load
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
index 8259edc..ce70cf7 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
@@ -31,15 +31,16 @@
 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.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeFileEnlistmentOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexFileEnlistmentOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -100,18 +101,19 @@
 
         // create factories and providers for B-Tree
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
-        IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
-        IIndexRegistryProvider<BTree> btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
+        ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
+        IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
         comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
-        BTreeFileEnlistmentOperatorDescriptor fileEnlistmentOp = new BTreeFileEnlistmentOperatorDescriptor(spec,
-                recDesc, storageManager, btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory,
-                leafFrameFactory, typeTraits, comparatorFactories);
+        ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+        TreeIndexFileEnlistmentOperatorDescriptor fileEnlistmentOp = new TreeIndexFileEnlistmentOperatorDescriptor(spec,
+                recDesc, storageManager, treeIndexRegistryProvider, btreeSplitProvider, interiorFrameFactory,
+                leafFrameFactory, typeTraits, comparatorFactories, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, fileEnlistmentOp, splitNCs);
 
         spec.addRoot(fileEnlistmentOp);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 7453c28..cc4e984 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -37,15 +37,16 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
+import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -102,9 +103,9 @@
 
         // create factories and providers for B-Tree
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
-        IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
-        IIndexRegistryProvider<BTree> btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
+        ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
+        IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples coming out of primary index
@@ -145,9 +146,10 @@
                                      // into search op
 
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
+        ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
         BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
-                btreeRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
-                comparatorFactories, true, lowKeyFields, highKeyFields, true, true);
+                treeIndexRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
+                comparatorFactories, true, lowKeyFields, highKeyFields, true, true, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
 
         // have each node print the results of its respective B-Tree
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index a108c91..6fbf53c 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -34,16 +34,17 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDiskOrderScanOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDiskOrderScanOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -96,8 +97,8 @@
         JobSpecification spec = new JobSpecification();
 
         String[] splitNCs = options.ncs.split(",");
-
-        IIndexRegistryProvider<BTree> btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
+        
+        IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples that we are retrieving from the primary index
@@ -119,14 +120,15 @@
 
         // create factories and providers for primary B-Tree
         TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
-        IBTreeInteriorFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
-        IBTreeLeafFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
+        ITreeIndexFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
+        ITreeIndexFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
 
         // use a disk-order scan to read primary index
         IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
-        BTreeDiskOrderScanOperatorDescriptor btreeScanOp = new BTreeDiskOrderScanOperatorDescriptor(spec, recDesc,
-                storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
-                primaryLeafFrameFactory, primaryTypeTraits);
+        ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+        TreeIndexDiskOrderScanOperatorDescriptor btreeScanOp = new TreeIndexDiskOrderScanOperatorDescriptor(spec, recDesc,
+                storageManager, treeIndexRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
+                primaryLeafFrameFactory, primaryTypeTraits, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, btreeScanOp, splitNCs);
 
         // sort the tuples as preparation for bulk load into secondary index
@@ -148,17 +150,17 @@
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
-        IBTreeInteriorFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
+        ITreeIndexFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
                 secondaryTupleWriterFactory);
-        IBTreeLeafFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(secondaryTupleWriterFactory);
+        ITreeIndexFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(secondaryTupleWriterFactory);
 
         // the B-Tree expects its keyfields to be at the front of its input
         // tuple
         int[] fieldPermutation = { 1, 0 };
         IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.secondaryBTreeName);
-        BTreeBulkLoadOperatorDescriptor btreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(spec, storageManager,
-                btreeRegistryProvider, btreeSplitProvider, secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
-                secondaryTypeTraits, comparatorFactories, fieldPermutation, 0.7f);
+        TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec, storageManager,
+                treeIndexRegistryProvider, btreeSplitProvider, secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
+                secondaryTypeTraits, comparatorFactories, fieldPermutation, 0.7f, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
 
         // connect the ops
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 7b0e714..e39f359 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -37,15 +37,16 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.examples.btree.helper.BTreeRegistryProvider;
 import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
+import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -96,7 +97,7 @@
 
         String[] splitNCs = options.ncs.split(",");
 
-        IIndexRegistryProvider<BTree> btreeRegistryProvider = BTreeRegistryProvider.INSTANCE;
+        IIndexRegistryProvider<ITreeIndex> btreeRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
 
         // schema of tuples coming out of secondary index
@@ -110,9 +111,9 @@
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(secondaryTypeTraits);
-        IBTreeInteriorFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
+        ITreeIndexFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
                 secondaryTupleWriterFactory);
-        IBTreeLeafFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(secondaryTupleWriterFactory);
+        ITreeIndexFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(secondaryTupleWriterFactory);
 
         // schema of tuples coming out of primary index
         RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
@@ -128,8 +129,8 @@
 
         // create factories and providers for secondary B-Tree
         TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
-        IBTreeInteriorFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
-        IBTreeLeafFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
+        ITreeIndexFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(primaryTupleWriterFactory);
+        ITreeIndexFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(primaryTupleWriterFactory);
 
         // comparators for btree, note that we only need a comparator for the
         // non-unique key
@@ -170,10 +171,11 @@
 
         IFileSplitProvider secondarySplitProvider = JobHelper.createFileSplitProvider(splitNCs,
                 options.secondaryBTreeName);
+        ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
                 storageManager, btreeRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
                 secondaryLeafFrameFactory, secondaryTypeTraits, comparatorFactories, true, secondaryLowKeyFields,
-                secondaryHighKeyFields, true, true);
+                secondaryHighKeyFields, true, true, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
 
         // secondary index will output tuples with [UTF8String, Integer]
@@ -189,7 +191,7 @@
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
                 primaryLeafFrameFactory, primaryTypeTraits, comparatorFactories, true, primaryLowKeyFields,
-                primaryHighKeyFields, true, true);
+                primaryHighKeyFields, true, true, opHelperFactory);
         JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
 
         // have each node print the results of its respective B-Tree
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 282621f..9fd7df4 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -17,7 +17,7 @@
 
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -30,7 +30,7 @@
 import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
 
 public class RuntimeContext {
-    private IndexRegistry<BTree> btreeRegistry;
+    private IndexRegistry<ITreeIndex> treeIndexRegistry;
     private IBufferCache bufferCache;
     private IFileMapManager fileMapManager;
 
@@ -39,7 +39,7 @@
         ICacheMemoryAllocator allocator = new HeapBufferAllocator();
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
         bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs, fileMapManager, 32768, 50, 100);
-        btreeRegistry = new IndexRegistry<BTree>();
+        treeIndexRegistry = new IndexRegistry<ITreeIndex>();
     }
 
     public void close() {
@@ -54,8 +54,8 @@
         return fileMapManager;
     }
 
-    public IndexRegistry<BTree> getBTreeRegistry() {
-        return btreeRegistry;
+    public IndexRegistry<ITreeIndex> getTreeIndexRegistry() {
+        return treeIndexRegistry;
     }
     
     public static RuntimeContext get(IHyracksStageletContext ctx) {
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
similarity index 69%
rename from hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
rename to hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
index 7bfb11b..6a6ddf8 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
@@ -16,20 +16,20 @@
 package edu.uci.ics.hyracks.examples.btree.helper;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 
-public class BTreeRegistryProvider implements IIndexRegistryProvider<BTree> {
+public class TreeIndexRegistryProvider implements IIndexRegistryProvider<ITreeIndex> {
     private static final long serialVersionUID = 1L;
 
-    public static final BTreeRegistryProvider INSTANCE = new BTreeRegistryProvider();
+    public static final TreeIndexRegistryProvider INSTANCE = new TreeIndexRegistryProvider();
 
-    private BTreeRegistryProvider() {
+    private TreeIndexRegistryProvider() {
     }
 
     @Override
-    public IndexRegistry<BTree> getRegistry(IHyracksStageletContext ctx) {
-        return RuntimeContext.get(ctx).getBTreeRegistry();
+    public IndexRegistry<ITreeIndex> getRegistry(IHyracksStageletContext ctx) {
+        return RuntimeContext.get(ctx).getTreeIndexRegistry();
     }
 }
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
index 2ffc96c..9f88abf 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeOperatorsTest.java
@@ -47,32 +47,34 @@
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeStatsOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexStatsOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestBTreeRegistryProvider;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
 
 public class BTreeOperatorsTest extends AbstractIntegrationTest {
 	static {
 		TestStorageManagerComponentHolder.init(8192, 20, 20);
 	}
-
+	
 	private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
-	private IIndexRegistryProvider<BTree> btreeRegistryProvider = new TestBTreeRegistryProvider();
-
+	private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
+	private ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+	
 	private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
 			"ddMMyy-hhmmssSS");
 	private final static String sep = System.getProperty("file.separator");
@@ -84,9 +86,9 @@
 	private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
 	private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
 			primaryTypeTraits);
-	private IBTreeInteriorFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(
+	private ITreeIndexFrameFactory primaryInteriorFrameFactory = new NSMInteriorFrameFactory(
 			primaryTupleWriterFactory);
-	private IBTreeLeafFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(
+	private ITreeIndexFrameFactory primaryLeafFrameFactory = new NSMLeafFrameFactory(
 			primaryTupleWriterFactory);
 
 	private static String primaryBtreeName = "primary"
@@ -114,9 +116,9 @@
 	private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
 	private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
 			secondaryTypeTraits);
-	private IBTreeInteriorFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
+	private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new NSMInteriorFrameFactory(
 			secondaryTupleWriterFactory);
-	private IBTreeLeafFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(
+	private ITreeIndexFrameFactory secondaryLeafFrameFactory = new NSMLeafFrameFactory(
 			secondaryTupleWriterFactory);
 
 	private static String secondaryBtreeName = "secondary"
@@ -197,11 +199,11 @@
 				NC1_ID);
 
 		int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
-		BTreeBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(
-				spec, storageManager, btreeRegistryProvider,
+		TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
+				spec, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
-				primaryComparatorFactories, fieldPermutation, 0.7f);
+				primaryComparatorFactories, fieldPermutation, 0.7f, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeBulkLoad, NC1_ID);
 
@@ -219,11 +221,11 @@
 	public void showPrimaryIndexStats() throws Exception {
 		JobSpecification spec = new JobSpecification();
 		
-		BTreeStatsOperatorDescriptor primaryStatsOp = new BTreeStatsOperatorDescriptor(
-				spec, storageManager, btreeRegistryProvider,
+		TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(
+				spec, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
-				primaryComparatorFactories);
+				primaryComparatorFactories, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryStatsOp, NC1_ID);
 		
@@ -258,11 +260,11 @@
 		int[] highKeyFields = null; // + infinity
 
 		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, primaryRecDesc, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
 				primaryComparatorFactories, true, lowKeyFields, highKeyFields,
-				true, true);
+				true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeSearchOp, NC1_ID);
 
@@ -311,11 +313,11 @@
 		int[] highKeyFields = { 1 };
 
 		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, primaryRecDesc, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
 				primaryComparatorFactories, true, lowKeyFields, highKeyFields,
-				true, true);
+				true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeSearchOp, NC1_ID);
 
@@ -360,11 +362,11 @@
 
 		// scan primary index
 		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, primaryRecDesc, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
 				primaryComparatorFactories, true, lowKeyFields, highKeyFields,
-				true, true);
+				true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeSearchOp, NC1_ID);
 
@@ -380,11 +382,11 @@
 
 		// load secondary index
 		int[] fieldPermutation = { 3, 0 };
-		BTreeBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new BTreeBulkLoadOperatorDescriptor(
-				spec, storageManager, btreeRegistryProvider,
+		TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
+				spec, storageManager, treeIndexRegistryProvider,
 				secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
 				secondaryLeafFrameFactory, secondaryTypeTraits,
-				secondaryComparatorFactories, fieldPermutation, 0.7f);
+				secondaryComparatorFactories, fieldPermutation, 0.7f, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				secondaryBtreeBulkLoad, NC1_ID);
 
@@ -432,11 +434,11 @@
 
 		// search secondary index
 		BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, secondaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, secondaryRecDesc, storageManager, treeIndexRegistryProvider,
 				secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
 				secondaryLeafFrameFactory, secondaryTypeTraits,
 				secondaryComparatorFactories, true, secondaryLowKeyFields,
-				secondaryHighKeyFields, true, true);
+				secondaryHighKeyFields, true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				secondaryBtreeSearchOp, NC1_ID);
 
@@ -447,11 +449,11 @@
 
 		// search primary index
 		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, primaryRecDesc, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
 				primaryComparatorFactories, true, primaryLowKeyFields,
-				primaryHighKeyFields, true, true);
+				primaryHighKeyFields, true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeSearchOp, NC1_ID);
 
@@ -509,23 +511,23 @@
 
 		// insert into primary index
 		int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
-		BTreeInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new BTreeInsertUpdateDeleteOperatorDescriptor(
-				spec, ordersDesc, storageManager, btreeRegistryProvider,
+		TreeIndexInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+				spec, ordersDesc, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
 				primaryComparatorFactories, primaryFieldPermutation,
-				TreeIndexOp.TI_INSERT);
+				IndexOp.INSERT, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeInsertOp, NC1_ID);
 
 		// first secondary index
 		int[] fieldPermutationB = { 4, 0 };
-		BTreeInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new BTreeInsertUpdateDeleteOperatorDescriptor(
-				spec, ordersDesc, storageManager, btreeRegistryProvider,
+		TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+				spec, ordersDesc, storageManager, treeIndexRegistryProvider,
 				secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
 				secondaryLeafFrameFactory, secondaryTypeTraits,
 				secondaryComparatorFactories, fieldPermutationB,
-				TreeIndexOp.TI_INSERT);
+				IndexOp.INSERT, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				secondaryInsertOp, NC1_ID);
 
@@ -580,11 +582,11 @@
 
 		// search secondary index
 		BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, secondaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, secondaryRecDesc, storageManager, treeIndexRegistryProvider,
 				secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
 				secondaryLeafFrameFactory, secondaryTypeTraits,
 				secondaryComparatorFactories, true, secondaryLowKeyFields,
-				secondaryHighKeyFields, true, true);
+				secondaryHighKeyFields, true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				secondaryBtreeSearchOp, NC1_ID);
 
@@ -595,11 +597,11 @@
 
 		// search primary index
 		BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
-				spec, primaryRecDesc, storageManager, btreeRegistryProvider,
+				spec, primaryRecDesc, storageManager, treeIndexRegistryProvider,
 				primaryBtreeSplitProvider, primaryInteriorFrameFactory,
 				primaryLeafFrameFactory, primaryTypeTraits,
 				primaryComparatorFactories, true, primaryLowKeyFields,
-				primaryHighKeyFields, true, true);
+				primaryHighKeyFields, true, true, opHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBtreeSearchOp, NC1_ID);
 
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrameFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrameFactory.java
deleted file mode 100644
index bddd636..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeInteriorFrameFactory.java
+++ /dev/null
@@ -1,22 +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.storage.am.btree.api;
-
-import java.io.Serializable;
-
-public interface IBTreeInteriorFrameFactory extends Serializable {
-    public IBTreeInteriorFrame getFrame();
-}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrameFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrameFactory.java
deleted file mode 100644
index 50733de..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeLeafFrameFactory.java
+++ /dev/null
@@ -1,22 +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.storage.am.btree.api;
-
-import java.io.Serializable;
-
-public interface IBTreeLeafFrameFactory extends Serializable {
-    public IBTreeLeafFrame getFrame();
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/AbstractBTreeOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/AbstractBTreeOperatorDescriptor.java
deleted file mode 100644
index 137eb77..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/AbstractBTreeOperatorDescriptor.java
+++ /dev/null
@@ -1,103 +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.storage.am.btree.dataflow;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public abstract class AbstractBTreeOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor implements
-        IBTreeOperatorDescriptorHelper {
-
-    private static final long serialVersionUID = 1L;
-
-    protected final IFileSplitProvider fileSplitProvider;
-
-    protected final IBinaryComparatorFactory[] comparatorFactories;
-
-    protected final IBTreeInteriorFrameFactory interiorFrameFactory;
-    protected final IBTreeLeafFrameFactory leafFrameFactory;
-
-    protected final IStorageManagerInterface storageManager;
-    protected final IIndexRegistryProvider<BTree> btreeRegistryProvider;
-
-    protected final ITypeTrait[] typeTraits;
-
-    public AbstractBTreeOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
-            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<BTree> btreeRegistryProvider, IFileSplitProvider fileSplitProvider,
-            IBTreeInteriorFrameFactory interiorFactory, IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories) {
-        super(spec, inputArity, outputArity);
-        this.fileSplitProvider = fileSplitProvider;
-        this.storageManager = storageManager;
-        this.btreeRegistryProvider = btreeRegistryProvider;
-        this.interiorFrameFactory = interiorFactory;
-        this.leafFrameFactory = leafFactory;
-        this.typeTraits = typeTraits;
-        this.comparatorFactories = comparatorFactories;
-        if (outputArity > 0)
-            recordDescriptors[0] = recDesc;
-    }
-
-    @Override
-    public IFileSplitProvider getBTreeFileSplitProvider() {
-        return fileSplitProvider;
-    }
-
-    @Override
-    public IBinaryComparatorFactory[] getBTreeComparatorFactories() {
-        return comparatorFactories;
-    }
-
-    @Override
-    public ITypeTrait[] getBTreeTypeTraits() {
-        return typeTraits;
-    }
-
-    @Override
-    public IBTreeInteriorFrameFactory getBTreeInteriorFactory() {
-        return interiorFrameFactory;
-    }
-
-    @Override
-    public IBTreeLeafFrameFactory getBTreeLeafFactory() {
-        return leafFrameFactory;
-    }
-
-    @Override
-    public IStorageManagerInterface getStorageManager() {
-        return storageManager;
-    }
-
-    @Override
-    public IIndexRegistryProvider<BTree> getBTreeRegistryProvider() {
-        return btreeRegistryProvider;
-    }
-
-    @Override
-    public RecordDescriptor getRecordDescriptor() {
-        return recordDescriptors[0];
-    }
-}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
index 5e723fb..c85c6a0 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
@@ -1,178 +1,32 @@
-/*
- * 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.btree.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-public final class BTreeOpHelper {   
-    
-	private IBTreeInteriorFrame interiorFrame;
-	private IBTreeLeafFrame leafFrame;
+public class BTreeOpHelper extends TreeIndexOpHelper {
 
-	private BTree btree;
-	private int btreeFileId = -1;
-	private int partition;
-
-    private IBTreeOperatorDescriptorHelper opDesc;
-    private IHyracksStageletContext ctx;
-
-	private IndexHelperOpenMode mode;
-
-    public BTreeOpHelper(IBTreeOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx, int partition,
+    public BTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksStageletContext ctx, int partition,
             IndexHelperOpenMode mode) {
-        this.opDesc = opDesc;
-        this.ctx = ctx;
-        this.mode = mode;
-        this.partition = partition;
+        super(opDesc, ctx, partition, mode);
     }
-
-	public void init() throws HyracksDataException {
+    
+    public ITreeIndex createTreeIndex() throws HyracksDataException {
         IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
-        IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
-        IFileSplitProvider fileSplitProvider = opDesc.getBTreeFileSplitProvider();
-
-        FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-        boolean fileIsMapped = fileMapProvider.isMapped(f);
-
-		switch (mode) {
-		
-		case OPEN: {
-			if (!fileIsMapped) {
-				throw new HyracksDataException(
-						"Trying to open btree from unmapped file " + f.toString());
-			}
-		}
-		break;
-
-		case CREATE:
-		case ENLIST: {
-			if (!fileIsMapped) {
-				bufferCache.createFile(f);
-			}
-		}
-		break;
-		
-		}
-		
-        int fileId = fileMapProvider.lookupFileId(f);		
-        try {
-        	bufferCache.openFile(fileId);
-        } catch(HyracksDataException e) {
-        	// revert state of buffer cache since file failed to open
-        	if(!fileIsMapped) {
-        		bufferCache.deleteFile(fileId);
-        	}
-        	throw e;
-        }
-        
-        // only set btreeFileId member when openFile() succeeds, 
-        // otherwise deinit() will try to close the file that failed to open
-        btreeFileId = fileId;
-
-		interiorFrame = opDesc.getBTreeInteriorFactory().getFrame();
-		leafFrame = opDesc.getBTreeLeafFactory().getFrame();
-
-        IndexRegistry<BTree> btreeRegistry = opDesc.getBTreeRegistryProvider().getRegistry(ctx);
-        btree = btreeRegistry.get(btreeFileId);
-        if (btree == null) {
-
-			// create new btree and register it
-			btreeRegistry.lock();
-			try {
-				// check if btree has already been registered by another thread
-				btree = btreeRegistry.get(btreeFileId);
-				if (btree == null) {
-					// this thread should create and register the btree
-
-					IBinaryComparator[] comparators = new IBinaryComparator[opDesc
-							.getBTreeComparatorFactories().length];
-					for (int i = 0; i < opDesc.getBTreeComparatorFactories().length; i++) {
-						comparators[i] = opDesc.getBTreeComparatorFactories()[i]
-								.createBinaryComparator();
-					}
-
-					MultiComparator cmp = new MultiComparator(opDesc
-							.getBTreeTypeTraits(), comparators);
-
-					// TODO: abstract away in some kind of factory
-					ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-					IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaDataFrameFactory);
-					btree = new BTree(bufferCache, freePageManager, opDesc.getBTreeInteriorFactory(),
-							opDesc.getBTreeLeafFactory(), cmp);
-					if (mode == IndexHelperOpenMode.CREATE) {
-						ITreeIndexMetaDataFrame metaFrame = btree.getFreePageManager().getMetaDataFrameFactory().getFrame();
-						try {
-							btree.create(btreeFileId, leafFrame, metaFrame);							
-						} catch (Exception e) {
-							throw new HyracksDataException(e);
-						}
-					}
-					btree.open(btreeFileId);
-					btreeRegistry.register(btreeFileId, btree);
-				}
-			} finally {
-				btreeRegistry.unlock();
-			}
-		}
-	}
-
-    public void deinit() throws HyracksDataException {
-        if (btreeFileId != -1) {
-            IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
-            bufferCache.closeFile(btreeFileId);
-        }
+        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0, metaDataFrameFactory);
+        return new BTree(bufferCache, freePageManager, opDesc.getTreeIndexInteriorFactory(),
+                opDesc.getTreeIndexLeafFactory(), cmp);
     }
-
-	public BTree getBTree() {
-		return btree;
-	}
-
-    public IHyracksStageletContext getHyracksStageletContext() {
-        return ctx;
-    }
-
-	public IBTreeOperatorDescriptorHelper getOperatorDescriptor() {
-		return opDesc;
-	}
-
-	public IBTreeLeafFrame getLeafFrame() {
-		return leafFrame;
-	}
-
-	public IBTreeInteriorFrame getInteriorFrame() {
-		return interiorFrame;
-	}
-
-	public int getBTreeFileId() {
-		return btreeFileId;
-	}
-}
\ No newline at end of file
+    
+    
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
new file mode 100644
index 0000000..f784624
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
+
+public class BTreeOpHelperFactory implements ITreeIndexOpHelperFactory {
+    
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
+            IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode) {        
+        return new BTreeOpHelper(opDesc, ctx, partition, mode);
+    }
+    
+    public ITreeIndexCursor createDiskOrderScanCursor(ITreeIndexFrame leafFrame) throws HyracksDataException {
+        return new BTreeDiskOrderScanCursor((IBTreeLeafFrame)leafFrame);
+    }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 6e13941..254935b 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -24,13 +24,14 @@
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
-public class BTreeSearchOperatorDescriptor extends AbstractBTreeOperatorDescriptor {
+public class BTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
@@ -42,13 +43,13 @@
     private boolean highKeyInclusive;
 
     public BTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<BTree> btreeRegistryProvider,
-            IFileSplitProvider fileSplitProvider, IBTreeInteriorFrameFactory interiorFactory,
-            IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits,
+            IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
             IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive) {
-        super(spec, 1, 1, recDesc, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFactory,
-                leafFactory, typeTraits, comparatorFactories);
+            boolean lowKeyInclusive, boolean highKeyInclusive, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                leafFrameFactory, typeTraits, comparatorFactories, opHelperFactory);
         this.isForward = isForward;
         this.lowKeyFields = lowKeyFields;
         this.highKeyFields = highKeyFields;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index a9729ed..9d33bb2 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -28,18 +28,22 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 
 public class BTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private BTreeOpHelper btreeOpHelper;
+    private TreeIndexOpHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
 
     private ByteBuffer writeBuffer;
@@ -56,16 +60,16 @@
     private RangePredicate rangePred;
     private MultiComparator lowKeySearchCmp;
     private MultiComparator highKeySearchCmp;
-    private IBTreeCursor cursor;
-    private IBTreeLeafFrame cursorFrame;
+    private ITreeIndexCursor cursor;
+    private ITreeIndexFrame cursorFrame;
     private BTreeOpContext opCtx;
 
     private RecordDescriptor recDesc;
 
-    public BTreeSearchOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
             int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
-        btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
         this.isForward = isForward;
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
@@ -82,17 +86,17 @@
 
     @Override
     public void open() throws HyracksDataException {
-        AbstractBTreeOperatorDescriptor opDesc = (AbstractBTreeOperatorDescriptor) btreeOpHelper
+        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
 
-        cursorFrame = opDesc.getBTreeLeafFactory().getFrame();
-        cursor = new RangeSearchCursor(cursorFrame);
+        cursorFrame = opDesc.getTreeIndexLeafFactory().createFrame();
+        cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame)cursorFrame);
 
         try {
 
-            btreeOpHelper.init();
-            btree = btreeOpHelper.getBTree();
+            treeIndexOpHelper.init();
+            btree = (BTree)treeIndexOpHelper.getTreeIndex();
 
             // construct range predicate
 
@@ -124,19 +128,19 @@
             rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
 
-            accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+            accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
 
-            writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
+            writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
             tb = new ArrayTupleBuilder(btree.getMultiComparator().getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext().getFrameSize());
+            appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext().getFrameSize());
             appender.reset(writeBuffer, true);
 
-            opCtx = btree.createOpContext(TreeIndexOp.TI_SEARCH, btreeOpHelper.getLeafFrame(), btreeOpHelper
+            opCtx = btree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(), treeIndexOpHelper
                     .getInteriorFrame(), null);
 
         } catch (Exception e) {
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
         }
     }
 
@@ -197,7 +201,7 @@
                 throw new HyracksDataException(e);
             }
         } finally {
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
         }
     }
 
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeStatsOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeStatsOperatorDescriptor.java
deleted file mode 100644
index 61bc973..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeStatsOperatorDescriptor.java
+++ /dev/null
@@ -1,34 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class BTreeStatsOperatorDescriptor extends AbstractBTreeOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-    
-    public BTreeStatsOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<BTree> btreeRegistryProvider, IFileSplitProvider fileSplitProvider,
-            IBTreeInteriorFrameFactory interiorFactory, IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories) {
-        super(spec, 0, 0, null, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFactory, leafFactory,
-                typeTraits, comparatorFactories);
-    }
-    
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeStatsOperatorNodePushable(this, ctx, partition);
-    }
-}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeOperatorDescriptorHelper.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeOperatorDescriptorHelper.java
deleted file mode 100644
index 522076f..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeOperatorDescriptorHelper.java
+++ /dev/null
@@ -1,30 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public interface IBTreeOperatorDescriptorHelper extends IActivityNode {
-    public IFileSplitProvider getBTreeFileSplitProvider();
-
-    public IBinaryComparatorFactory[] getBTreeComparatorFactories();
-
-    public ITypeTrait[] getBTreeTypeTraits();
-
-    public IBTreeInteriorFrameFactory getBTreeInteriorFactory();
-
-    public IBTreeLeafFrameFactory getBTreeLeafFactory();
-
-    public IStorageManagerInterface getStorageManager();
-
-    public IIndexRegistryProvider<BTree> getBTreeRegistryProvider();
-
-    public RecordDescriptor getRecordDescriptor();
-}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/FieldPrefixNSMLeafFrameFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/FieldPrefixNSMLeafFrameFactory.java
index b6665c1..22da2f5 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/FieldPrefixNSMLeafFrameFactory.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/FieldPrefixNSMLeafFrameFactory.java
@@ -16,10 +16,10 @@
 package edu.uci.ics.hyracks.storage.am.btree.frames;
 
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
 
-public class FieldPrefixNSMLeafFrameFactory implements IBTreeLeafFrameFactory {
+public class FieldPrefixNSMLeafFrameFactory implements ITreeIndexFrameFactory {
 
     private static final long serialVersionUID = 1L;
     private ITreeIndexTupleWriterFactory tupleWriterFactory;
@@ -29,7 +29,7 @@
     }
 
     @Override
-    public IBTreeLeafFrame getFrame() {
+    public IBTreeLeafFrame createFrame() {
         return new FieldPrefixNSMLeafFrame(tupleWriterFactory.createTupleWriter());
     }
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMInteriorFrameFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMInteriorFrameFactory.java
index b201cd1..0d93ed0 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMInteriorFrameFactory.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMInteriorFrameFactory.java
@@ -16,10 +16,10 @@
 package edu.uci.ics.hyracks.storage.am.btree.frames;
 
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
 
-public class NSMInteriorFrameFactory implements IBTreeInteriorFrameFactory {
+public class NSMInteriorFrameFactory implements ITreeIndexFrameFactory {
 
     private static final long serialVersionUID = 1L;
     private ITreeIndexTupleWriterFactory tupleWriterFactory;
@@ -29,7 +29,7 @@
     }
 
     @Override
-    public IBTreeInteriorFrame getFrame() {
+    public IBTreeInteriorFrame createFrame() {
         return new NSMInteriorFrame(tupleWriterFactory.createTupleWriter());
     }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMLeafFrameFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMLeafFrameFactory.java
index f7c8c11..d8f401f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMLeafFrameFactory.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/NSMLeafFrameFactory.java
@@ -16,10 +16,10 @@
 package edu.uci.ics.hyracks.storage.am.btree.frames;
 
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
 
-public class NSMLeafFrameFactory implements IBTreeLeafFrameFactory {
+public class NSMLeafFrameFactory implements ITreeIndexFrameFactory {
 
     private static final long serialVersionUID = 1L;
     private ITreeIndexTupleWriterFactory tupleWriterFactory;
@@ -29,7 +29,7 @@
     }
 
     @Override
-    public IBTreeLeafFrame getFrame() {
+    public IBTreeLeafFrame createFrame() {
         return new NSMLeafFrame(tupleWriterFactory.createTupleWriter());
     }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 8b02ff3..3c9a67a 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -22,25 +22,28 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOpContext;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
-public class BTree {
+public class BTree implements ITreeIndex {
 
 	public static final float DEFAULT_FILL_FACTOR = 0.7f;
 
@@ -57,8 +60,8 @@
 
 	private final IBufferCache bufferCache;
 	private int fileId;
-	private final IBTreeInteriorFrameFactory interiorFrameFactory;
-	private final IBTreeLeafFrameFactory leafFrameFactory;
+	private final ITreeIndexFrameFactory interiorFrameFactory;
+	private final ITreeIndexFrameFactory leafFrameFactory;
 	private final MultiComparator cmp;
 	private final ReadWriteLock treeLatch;
 	private final RangePredicate diskOrderScanPredicate;
@@ -103,8 +106,8 @@
 	}
 
 	public BTree(IBufferCache bufferCache, IFreePageManager freePageManager,
-			IBTreeInteriorFrameFactory interiorFrameFactory,
-			IBTreeLeafFrameFactory leafFrameFactory, MultiComparator cmp) {
+	        ITreeIndexFrameFactory interiorFrameFactory,
+	        ITreeIndexFrameFactory leafFrameFactory, MultiComparator cmp) {
 		this.bufferCache = bufferCache;
 		this.interiorFrameFactory = interiorFrameFactory;
 		this.leafFrameFactory = leafFrameFactory;
@@ -115,7 +118,8 @@
 				true, true, cmp, cmp);
 	}
 
-	public void create(int fileId, IBTreeLeafFrame leafFrame,
+	@Override
+	public void create(int fileId, ITreeIndexFrame leafFrame,
 			ITreeIndexMetaDataFrame metaFrame) throws Exception {
 
 		if (created)
@@ -240,10 +244,13 @@
 		}
 	}
 
-	public void diskOrderScan(DiskOrderScanCursor cursor,
-			IBTreeLeafFrame leafFrame, ITreeIndexMetaDataFrame metaFrame)
+	@Override
+	public void diskOrderScan(ITreeIndexCursor icursor,
+			ITreeIndexFrame leafFrame, ITreeIndexMetaDataFrame metaFrame)
 			throws HyracksDataException {
-		int currentPageId = rootPage + 1;
+	    BTreeDiskOrderScanCursor cursor = (BTreeDiskOrderScanCursor)icursor;
+	    
+	    int currentPageId = rootPage + 1;
 		int maxPageId = freePageManager.getMaxPage(metaFrame);
 
 		ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(
@@ -256,7 +263,7 @@
 		cursor.open(page, diskOrderScanPredicate);
 	}
 
-	public void search(IBTreeCursor cursor, RangePredicate pred,
+	public void search(ITreeIndexCursor cursor, RangePredicate pred,
 			BTreeOpContext ctx) throws Exception {
 		ctx.reset();
 		ctx.pred = pred;
@@ -388,9 +395,11 @@
 		}
 	}
 
-	public void insert(ITupleReference tuple, BTreeOpContext ctx)
+	@Override
+	public void insert(ITupleReference tuple, IndexOpContext ictx)
 			throws Exception {
-		ctx.reset();
+		BTreeOpContext ctx = (BTreeOpContext) ictx;
+	    ctx.reset();
 		ctx.pred.setLowKeyComparator(cmp);
 		ctx.pred.setHighKeyComparator(cmp);
 		ctx.pred.setLowKey(tuple, true);
@@ -496,8 +505,8 @@
 					rightNode.acquireWriteLatch();
 					writeLatchesAcquired++;
 					try {
-						IBTreeLeafFrame rightFrame = leafFrameFactory
-								.getFrame();
+						IBTreeLeafFrame rightFrame = (IBTreeLeafFrame)leafFrameFactory
+								.createFrame();
 						rightFrame.setPage(rightNode);
 						rightFrame.initBuffer((byte) 0);
 						rightFrame.setPageTupleFieldCount(cmp.getFieldCount());
@@ -591,7 +600,7 @@
 			rightNode.acquireWriteLatch();
 			writeLatchesAcquired++;
 			try {
-				ITreeIndexFrame rightFrame = interiorFrameFactory.getFrame();
+				ITreeIndexFrame rightFrame = interiorFrameFactory.createFrame();
 				rightFrame.setPage(rightNode);
 				rightFrame.initBuffer((byte) ctx.interiorFrame.getLevel());
 				rightFrame.setPageTupleFieldCount(cmp.getKeyFieldCount());
@@ -649,9 +658,11 @@
 		}
 	}
 
-	public void delete(ITupleReference tuple, BTreeOpContext ctx)
+	@Override
+	public void delete(ITupleReference tuple, IndexOpContext ictx)
 			throws Exception {
-		ctx.reset();
+	    BTreeOpContext ctx = (BTreeOpContext)ictx;
+	    ctx.reset();
 		ctx.pred.setLowKeyComparator(cmp);
 		ctx.pred.setHighKeyComparator(cmp);
 		ctx.pred.setLowKey(tuple, true);
@@ -706,7 +717,7 @@
 
 		// will this leaf become empty?
 		if (ctx.leafFrame.getTupleCount() == 1) {
-			IBTreeLeafFrame siblingFrame = leafFrameFactory.getFrame();
+			IBTreeLeafFrame siblingFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
 
 			ICachedPage leftNode = null;
 			ICachedPage rightNode = null;
@@ -843,11 +854,11 @@
 		}
 	}
 
-	private final void acquireLatch(ICachedPage node, TreeIndexOp op,
+	private final void acquireLatch(ICachedPage node, IndexOp op,
 			boolean isLeaf) {
 		if (isLeaf
-				&& (op.equals(TreeIndexOp.TI_INSERT) || op
-						.equals(TreeIndexOp.TI_DELETE))) {
+				&& (op.equals(IndexOp.INSERT) || op
+						.equals(IndexOp.DELETE))) {
 			node.acquireWriteLatch();
 			writeLatchesAcquired++;
 		} else {
@@ -856,11 +867,11 @@
 		}
 	}
 
-	private final void releaseLatch(ICachedPage node, TreeIndexOp op,
+	private final void releaseLatch(ICachedPage node, IndexOp op,
 			boolean isLeaf) {
 		if (isLeaf
-				&& (op.equals(TreeIndexOp.TI_INSERT) || op
-						.equals(TreeIndexOp.TI_DELETE))) {
+				&& (op.equals(IndexOp.INSERT) || op
+						.equals(IndexOp.DELETE))) {
 			node.releaseWriteLatch();
 			writeLatchesReleased++;
 		} else {
@@ -958,7 +969,7 @@
 
 						switch (ctx.op) {
 
-						case TI_INSERT: {
+						case INSERT: {
 							if (ctx.splitKey.getBuffer() != null) {
 								node = bufferCache.pin(BufferedFileHandle
 										.getDiskPageId(fileId, pageId), false);
@@ -980,7 +991,7 @@
 						}
 							break;
 
-						case TI_DELETE: {
+						case DELETE: {
 							if (ctx.splitKey.getBuffer() != null) {
 								node = bufferCache.pin(BufferedFileHandle
 										.getDiskPageId(fileId, pageId), false);
@@ -1002,7 +1013,7 @@
 						}
 							break;
 
-						case TI_SEARCH: {
+						case SEARCH: {
 							// do nothing
 						}
 							break;
@@ -1038,17 +1049,17 @@
 				}
 			} else { // isLeaf and !smFlag
 				switch (ctx.op) {
-				case TI_INSERT: {
+				case INSERT: {
 					insertLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
 				}
 					break;
 
-				case TI_DELETE: {
+				case DELETE: {
 					deleteLeaf(node, pageId, ctx.pred.getLowKey(), ctx);
 				}
 					break;
 
-				case TI_SEARCH: {
+				case SEARCH: {
 					ctx.cursor.open(node, ctx.pred);
 				}
 					break;
@@ -1083,7 +1094,7 @@
 
 	private boolean bulkNewPage = false;
 
-	public final class BulkLoadContext {
+	public final class BulkLoadContext implements IIndexBulkLoadContext {
 		public final int slotSize;
 		public final int leafMaxBytes;
 		public final int interiorMaxBytes;
@@ -1205,24 +1216,27 @@
 	}
 
 	// assumes btree has been created and opened
-	public BulkLoadContext beginBulkLoad(float fillFactor,
-			IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
-			ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
-
+	@Override
+	public IIndexBulkLoadContext beginBulkLoad(float fillFactor,
+			ITreeIndexFrame leafFrame, ITreeIndexFrame interiorFrame,
+			ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {	    
+	    
 		if (loaded)
 			throw new HyracksDataException(
 					"Trying to bulk-load BTree but has BTree already been loaded.");
 
-		BulkLoadContext ctx = new BulkLoadContext(fillFactor, leafFrame,
-				interiorFrame, metaFrame);
+		BulkLoadContext ctx = new BulkLoadContext(fillFactor, (IBTreeLeafFrame)leafFrame,
+				(IBTreeInteriorFrame)interiorFrame, metaFrame);
 		ctx.nodeFrontiers.get(0).lastTuple.setFieldCount(cmp.getFieldCount());
 		ctx.splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
 		return ctx;
 	}
 
-	public void bulkLoadAddTuple(BulkLoadContext ctx, ITupleReference tuple)
+	@Override
+	public void bulkLoadAddTuple(IIndexBulkLoadContext ictx, ITupleReference tuple)
 			throws HyracksDataException {
-		NodeFrontier leafFrontier = ctx.nodeFrontiers.get(0);
+	    BulkLoadContext ctx = (BulkLoadContext)ictx;
+	    NodeFrontier leafFrontier = ctx.nodeFrontiers.get(0);
 		IBTreeLeafFrame leafFrame = ctx.leafFrame;
 
 		int spaceNeeded = ctx.tupleWriter.bytesRequired(tuple) + ctx.slotSize;
@@ -1276,8 +1290,10 @@
 		// System.out.println(s);
 	}
 
-	public void endBulkLoad(BulkLoadContext ctx) throws HyracksDataException {
+	@Override
+	public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
 		// copy root
+	    BulkLoadContext ctx = (BulkLoadContext)ictx;
 		ICachedPage rootNode = bufferCache.pin(BufferedFileHandle
 				.getDiskPageId(fileId, rootPage), bulkNewPage);
 		rootNode.acquireWriteLatch();
@@ -1311,18 +1327,18 @@
 		loaded = true;
 	}
 	
-	public BTreeOpContext createOpContext(TreeIndexOp op,
-			IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
+	@Override
+	public BTreeOpContext createOpContext(IndexOp op,
+			ITreeIndexFrame leafFrame, ITreeIndexFrame interiorFrame,
 			ITreeIndexMetaDataFrame metaFrame) {
-		// TODO: figure out better tree-height hint
-		return new BTreeOpContext(op, leafFrame, interiorFrame, metaFrame, 6);
+		return new BTreeOpContext(op, (IBTreeLeafFrame)leafFrame, (IBTreeInteriorFrame)interiorFrame, metaFrame, 6);
 	}
 
-	public IBTreeInteriorFrameFactory getInteriorFrameFactory() {
+	public ITreeIndexFrameFactory getInteriorFrameFactory() {
 		return interiorFrameFactory;
 	}
 
-	public IBTreeLeafFrameFactory getLeafFrameFactory() {
+	public ITreeIndexFrameFactory getLeafFrameFactory() {
 		return leafFrameFactory;
 	}
 
@@ -1336,5 +1352,20 @@
 	
 	public int getRootPageId() {
 	    return rootPage;
-	}
+	}  
+
+    @Override
+    public void update(ITupleReference tuple, IndexOpContext ictx) throws Exception {
+        throw new Exception("BTree Update not implemented.");
+    }
+
+    @Override
+    public int getFieldCount() {
+        return cmp.getFieldCount();
+    }
+
+    @Override
+    public IndexType getIndexType() {
+        return IndexType.BTREE;
+    }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeDiskOrderScanCursor.java
similarity index 94%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java
rename to hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeDiskOrderScanCursor.java
index 10eb042..c80a6d4 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/DiskOrderScanCursor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeDiskOrderScanCursor.java
@@ -16,16 +16,16 @@
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.ISearchPredicate;
+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.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
-public class DiskOrderScanCursor implements IBTreeCursor {
+public class BTreeDiskOrderScanCursor implements ITreeIndexCursor {
 
     // TODO: might want to return tuples in physical order, not logical order to
     // speed up access
@@ -41,7 +41,7 @@
 
     private ITreeIndexTupleReference frameTuple;
 
-    public DiskOrderScanCursor(IBTreeLeafFrame frame) {
+    public BTreeDiskOrderScanCursor(IBTreeLeafFrame frame) {
         this.frame = frame;
         this.frameTuple = frame.getTupleWriter().createTupleReference();
     }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
index 7d38dac..975a946 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
@@ -15,19 +15,20 @@
 
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.IntArrayList;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOpContext;
 
-public final class BTreeOpContext {
-    public final TreeIndexOp op;
+public final class BTreeOpContext implements IndexOpContext {
+    public final IndexOp op;
     public final IBTreeLeafFrame leafFrame;
     public final IBTreeInteriorFrame interiorFrame;
     public final ITreeIndexMetaDataFrame metaFrame;
-    public IBTreeCursor cursor;
+    public ITreeIndexCursor cursor;
     public RangePredicate pred;
     public final BTreeSplitKey splitKey;
     public int opRestarts = 0;
@@ -35,7 +36,7 @@
     public final IntArrayList smPages;
     public final IntArrayList freePages;
 
-    public BTreeOpContext(TreeIndexOp op, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
+    public BTreeOpContext(IndexOp op, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
     		ITreeIndexMetaDataFrame metaFrame, int treeHeightHint) {
         this.op = op;
         this.leafFrame = leafFrame;
@@ -43,7 +44,7 @@
         this.metaFrame = metaFrame;
 
         pageLsns = new IntArrayList(treeHeightHint, treeHeightHint);
-        if (op != TreeIndexOp.TI_SEARCH) {
+        if (op != IndexOp.SEARCH) {
             smPages = new IntArrayList(treeHeightHint, treeHeightHint);
             freePages = new IntArrayList(treeHeightHint, treeHeightHint);
             pred = new RangePredicate(true, null, null, true, true, null, null);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
similarity index 96%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java
rename to hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 2929726..a8d76c3 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangeSearchCursor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -17,9 +17,9 @@
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.ISearchPredicate;
+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.ITreeIndexTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleMode;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.FindTupleNoExactMatchPolicy;
@@ -28,7 +28,7 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
-public class RangeSearchCursor implements IBTreeCursor {
+public class BTreeRangeSearchCursor implements ITreeIndexCursor {
 
     private int fileId = -1;
     private ICachedPage page = null;
@@ -53,7 +53,7 @@
     private ITupleReference lowKey;
     private ITupleReference highKey;
 
-    public RangeSearchCursor(IBTreeLeafFrame frame) {
+    public BTreeRangeSearchCursor(IBTreeLeafFrame frame) {
         this.frame = frame;
         this.frameTuple = frame.createTupleReference();
     }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangePredicate.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangePredicate.java
index 0ed7f47..699f6a6 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangePredicate.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/RangePredicate.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.btree.impls;
 
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.btree.api.ISearchPredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
 public class RangePredicate implements ISearchPredicate {
diff --git a/hyracks-storage-am-common/pom.xml b/hyracks-storage-am-common/pom.xml
index db9ca5f..750b3e9 100644
--- a/hyracks-storage-am-common/pom.xml
+++ b/hyracks-storage-am-common/pom.xml
@@ -45,5 +45,12 @@
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-dataflow-std</artifactId>
+  		<version>0.1.5</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>  
   </dependencies>
 </project>
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoadContext.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoadContext.java
new file mode 100644
index 0000000..a896d80
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexBulkLoadContext.java
@@ -0,0 +1,4 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+public interface IIndexBulkLoadContext {
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/ISearchPredicate.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchPredicate.java
similarity index 93%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/ISearchPredicate.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchPredicate.java
index 255c370..71ab7af 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/ISearchPredicate.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ISearchPredicate.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.api;
+package edu.uci.ics.hyracks.storage.am.common.api;
 
 import java.io.Serializable;
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
new file mode 100644
index 0000000..0dea866
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
@@ -0,0 +1,62 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+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.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOpContext;
+
+public interface ITreeIndex {
+	// init:
+
+	public void create(int indexFileId, ITreeIndexFrame leafFrame,
+			ITreeIndexMetaDataFrame metaFrame) throws Exception;
+
+	public void open(int indexFileId);
+
+	// operations:
+
+	public void insert(ITupleReference tuple, IndexOpContext ictx)
+			throws Exception;
+
+	public void update(ITupleReference tuple, IndexOpContext ictx)
+			throws Exception;
+
+	public void delete(ITupleReference tuple, IndexOpContext ictx)
+			throws Exception;
+
+	public IndexOpContext createOpContext(IndexOp op,
+			ITreeIndexFrame leafFrame, ITreeIndexFrame interiorFrame,
+			ITreeIndexMetaDataFrame metaFrame);
+
+	// bulk loading:
+
+	public IIndexBulkLoadContext beginBulkLoad(float fillFactor,
+			ITreeIndexFrame leafFrame, ITreeIndexFrame interiorFrame,
+			ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException;
+
+	public void bulkLoadAddTuple(IIndexBulkLoadContext ictx,
+			ITupleReference tuple) throws HyracksDataException;
+
+	public void endBulkLoad(IIndexBulkLoadContext ictx)
+			throws HyracksDataException;
+
+	// search:
+	public void diskOrderScan(ITreeIndexCursor icursor, ITreeIndexFrame leafFrame,
+			ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException;
+	
+	
+
+	// utility:
+
+	public IFreePageManager getFreePageManager();
+
+	public int getRootPageId();
+
+	public ITreeIndexFrameFactory getLeafFrameFactory();
+
+	public ITreeIndexFrameFactory getInteriorFrameFactory();
+
+	public int getFieldCount();
+	
+	public IndexType getIndexType();
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeCursor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
similarity index 92%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeCursor.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
index 8934921..49f2d7f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/api/IBTreeCursor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
@@ -13,13 +13,13 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.api;
+package edu.uci.ics.hyracks.storage.am.common.api;
 
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 
-public interface IBTreeCursor {
+public interface ITreeIndexCursor {
     public void reset();
 
     public boolean hasNext() throws Exception;
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrameFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrameFactory.java
new file mode 100644
index 0000000..9ec69d9
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexFrameFactory.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+import java.io.Serializable;
+
+public interface ITreeIndexFrameFactory extends Serializable {
+	public ITreeIndexFrame createFrame();
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrameFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrameFactory.java
index 8c05637..07473ef 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrameFactory.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexMetaDataFrameFactory.java
@@ -17,5 +17,5 @@
 
 
 public interface ITreeIndexMetaDataFrameFactory {
-    public ITreeIndexMetaDataFrame getFrame();
+    public ITreeIndexMetaDataFrame createFrame();
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IndexType.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IndexType.java
new file mode 100644
index 0000000..6f83e0b
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IndexType.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.hyracks.storage.am.common.api;
+
+public enum IndexType {
+	BTREE, RTREE, INVERTED
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
new file mode 100644
index 0000000..0ab2a9a
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -0,0 +1,109 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public abstract class AbstractTreeIndexOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor implements
+        ITreeIndexOperatorDescriptorHelper {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final IFileSplitProvider fileSplitProvider;
+
+    protected final IBinaryComparatorFactory[] comparatorFactories;
+
+    protected final ITreeIndexFrameFactory interiorFrameFactory;
+    protected final ITreeIndexFrameFactory leafFrameFactory;
+
+    protected final IStorageManagerInterface storageManager;
+    protected final IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
+
+    protected final ITypeTrait[] typeTraits;
+
+    protected final ITreeIndexOpHelperFactory opHelperFactory;
+    
+    public AbstractTreeIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
+            RecordDescriptor recDesc, IStorageManagerInterface storageManager,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, inputArity, outputArity);
+        this.fileSplitProvider = fileSplitProvider;
+        this.storageManager = storageManager;
+        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
+        this.interiorFrameFactory = interiorFrameFactory;
+        this.leafFrameFactory = leafFrameFactory;
+        this.typeTraits = typeTraits;
+        this.comparatorFactories = comparatorFactories;
+        this.opHelperFactory = opHelperFactory;
+        if (outputArity > 0)
+            recordDescriptors[0] = recDesc;
+    }
+
+    @Override
+    public IFileSplitProvider getTreeIndexFileSplitProvider() {
+        return fileSplitProvider;
+    }
+
+    @Override
+    public IBinaryComparatorFactory[] getTreeIndexComparatorFactories() {
+        return comparatorFactories;
+    }
+
+    @Override
+    public ITypeTrait[] getTreeIndexTypeTraits() {
+        return typeTraits;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getTreeIndexInteriorFactory() {
+        return interiorFrameFactory;
+    }
+
+    @Override
+    public ITreeIndexFrameFactory getTreeIndexLeafFactory() {
+        return leafFrameFactory;
+    }
+
+    @Override
+    public IStorageManagerInterface getStorageManager() {
+        return storageManager;
+    }
+
+    @Override
+    public IIndexRegistryProvider<ITreeIndex> getTreeIndexRegistryProvider() {
+        return treeIndexRegistryProvider;
+    }
+
+    @Override
+    public RecordDescriptor getRecordDescriptor() {
+        return recordDescriptors[0];
+    }
+    
+    @Override
+    public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory() {
+    	return opHelperFactory;
+    }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
new file mode 100644
index 0000000..583ef7d
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+
+public interface ITreeIndexOpHelperFactory extends Serializable {
+	public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx, int partition,
+            IndexHelperOpenMode mode);
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
new file mode 100644
index 0000000..99affd3
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
+
+import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public interface ITreeIndexOperatorDescriptorHelper extends IActivityNode {
+    public IFileSplitProvider getTreeIndexFileSplitProvider();
+
+    public IBinaryComparatorFactory[] getTreeIndexComparatorFactories();
+
+    public ITypeTrait[] getTreeIndexTypeTraits();
+
+    public ITreeIndexFrameFactory getTreeIndexInteriorFactory();
+
+    public ITreeIndexFrameFactory getTreeIndexLeafFactory();
+
+    public IStorageManagerInterface getStorageManager();
+
+    public IIndexRegistryProvider<ITreeIndex> getTreeIndexRegistryProvider();
+
+    public RecordDescriptor getRecordDescriptor();
+    
+    public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory();
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/PermutingFrameTupleReference.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/PermutingFrameTupleReference.java
similarity index 96%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/PermutingFrameTupleReference.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/PermutingFrameTupleReference.java
index a3f343a..3db9db2 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/PermutingFrameTupleReference.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/PermutingFrameTupleReference.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
similarity index 60%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index 2f8e4a9..4398aad 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -23,25 +23,23 @@
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
-public class BTreeBulkLoadOperatorDescriptor extends AbstractBTreeOperatorDescriptor {
+public class TreeIndexBulkLoadOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     private final int[] fieldPermutation;
-    private final float fillFactor;
-
-    public BTreeBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<BTree> btreeRegistryProvider, IFileSplitProvider fileSplitProvider,
-            IBTreeInteriorFrameFactory interiorFactory, IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, float fillFactor) {
-        super(spec, 1, 0, null, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFactory, leafFactory,
-                typeTraits, comparatorFactories);
+    private final float fillFactor;    
+    
+    public TreeIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, float fillFactor, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, 1, 0, null, storageManager, treeIndexRegistryProvider, fileSplitProvider, interiorFrameFactory, leafFrameFactory,
+                typeTraits, comparatorFactories, opHelperFactory);
         this.fieldPermutation = fieldPermutation;
         this.fillFactor = fillFactor;
     }
@@ -49,7 +47,7 @@
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
+        return new TreeIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
                 recordDescProvider);
     }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
similarity index 65%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
index e13a82a..60e7e98 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.nio.ByteBuffer;
 
@@ -22,24 +22,23 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrame;
 
-public class BTreeBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+public class TreeIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
     private float fillFactor;
-    private final BTreeOpHelper btreeOpHelper;
+    private final TreeIndexOpHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
-    private BTree.BulkLoadContext bulkLoadCtx;
+    private IIndexBulkLoadContext bulkLoadCtx;
 
     private IRecordDescriptorProvider recordDescProvider;
 
     private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
 
-    public BTreeBulkLoadOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
-        btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
         this.fillFactor = fillFactor;
         this.recordDescProvider = recordDescProvider;
         tuple.setFieldPermutation(fieldPermutation);
@@ -47,19 +46,19 @@
 
     @Override
     public void open() throws HyracksDataException {
-        AbstractBTreeOperatorDescriptor opDesc = (AbstractBTreeOperatorDescriptor) btreeOpHelper
+        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
         ITreeIndexMetaDataFrame metaFrame = new LIFOMetaDataFrame();
         try {
-            btreeOpHelper.init();
-            btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
-            bulkLoadCtx = btreeOpHelper.getBTree().beginBulkLoad(fillFactor, btreeOpHelper.getLeafFrame(),
-                    btreeOpHelper.getInteriorFrame(), metaFrame);
+            treeIndexOpHelper.init();
+            treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
+            bulkLoadCtx = treeIndexOpHelper.getTreeIndex().beginBulkLoad(fillFactor, treeIndexOpHelper.getLeafFrame(),
+                    treeIndexOpHelper.getInteriorFrame(), metaFrame);
         } catch (Exception e) {
             // cleanup in case of failure
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
             throw new HyracksDataException(e);
         }
     }
@@ -70,16 +69,16 @@
         int tupleCount = accessor.getTupleCount();
         for (int i = 0; i < tupleCount; i++) {
             tuple.reset(accessor, i);
-            btreeOpHelper.getBTree().bulkLoadAddTuple(bulkLoadCtx, tuple);
+            treeIndexOpHelper.getTreeIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
         }
     }
 
     @Override
     public void close() throws HyracksDataException {
         try {
-            btreeOpHelper.getBTree().endBulkLoad(bulkLoadCtx);
+            treeIndexOpHelper.getTreeIndex().endBulkLoad(bulkLoadCtx);
         } finally {
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
         }
     }
 
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
similarity index 63%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index e684ff4..f02f10e 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -23,27 +23,25 @@
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
-public class BTreeDiskOrderScanOperatorDescriptor extends AbstractBTreeOperatorDescriptor {
+public class TreeIndexDiskOrderScanOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    public BTreeDiskOrderScanOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<BTree> btreeRegistryProvider,
-            IFileSplitProvider fileSplitProvider, IBTreeInteriorFrameFactory interiorFactory,
-            IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits) {
-        super(spec, 0, 1, recDesc, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFactory,
-                leafFactory, typeTraits, null);
+    public TreeIndexDiskOrderScanOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+            IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> btreeRegistryProvider,
+            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, 0, 1, recDesc, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                leafFrameFactory, typeTraits, null, opHelperFactory);
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeDiskOrderScanOperatorNodePushable(this, ctx, partition);
+        return new TreeIndexDiskOrderScanOperatorNodePushable(this, ctx, partition);
     }
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
similarity index 66%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 16a1d05..a8a1b64 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
@@ -24,40 +24,38 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.DiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
-public class BTreeDiskOrderScanOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-    private final BTreeOpHelper btreeOpHelper;
+public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+    private final TreeIndexOpHelper treeIndexOpHelper;
 
-    public BTreeDiskOrderScanOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public TreeIndexDiskOrderScanOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition) {
-        btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
     }
 
     @Override
     public void initialize() throws HyracksDataException {
 
-        IBTreeLeafFrame cursorFrame = btreeOpHelper.getOperatorDescriptor().getBTreeLeafFactory().getFrame();
-        DiskOrderScanCursor cursor = new DiskOrderScanCursor(cursorFrame);
+        ITreeIndexFrame cursorFrame = treeIndexOpHelper.getOperatorDescriptor().getTreeIndexLeafFactory().createFrame();
+        ITreeIndexCursor cursor = treeIndexOpHelper.createDiskOrderScanCursor(cursorFrame);
         ITreeIndexMetaDataFrame metaFrame = new LIFOMetaDataFrame();
 
         try {
         
-        	btreeOpHelper.init();
+        	treeIndexOpHelper.init();
         	
         	try {
-        		btreeOpHelper.getBTree().diskOrderScan(cursor, cursorFrame, metaFrame);
+        		treeIndexOpHelper.getTreeIndex().diskOrderScan(cursor, cursorFrame, metaFrame);
 
-        		MultiComparator cmp = btreeOpHelper.getBTree().getMultiComparator();
-        		ByteBuffer frame = btreeOpHelper.getHyracksStageletContext().allocateFrame();
-        		FrameTupleAppender appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext().getFrameSize());
+        		int fieldCount = treeIndexOpHelper.getTreeIndex().getFieldCount();
+        		ByteBuffer frame = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+        		FrameTupleAppender appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext().getFrameSize());
         		appender.reset(frame, true);
-        		ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
+        		ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         		DataOutput dos = tb.getDataOutput();
 
         		while (cursor.hasNext()) {
@@ -96,6 +94,6 @@
 
     @Override
     public void deinitialize() throws HyracksDataException {
-    	btreeOpHelper.deinit();
+    	treeIndexOpHelper.deinit();
     }
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
similarity index 69%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
index 0a41bd4..f5348c3 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -22,29 +22,28 @@
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
-public class BTreeDropOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+public class TreeIndexDropOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     private IStorageManagerInterface storageManager;
-    private IIndexRegistryProvider<BTree> btreeRegistryProvider;
+    private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
     private IFileSplitProvider fileSplitProvider;
     
-    public BTreeDropOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<BTree> btreeRegistryProvider, IFileSplitProvider fileSplitProvider) {
+    public TreeIndexDropOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider) {
         super(spec, 0, 0);
         this.storageManager = storageManager;
-        this.btreeRegistryProvider = btreeRegistryProvider;
+        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
         this.fileSplitProvider = fileSplitProvider;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeDropOperatorNodePushable(ctx, storageManager, btreeRegistryProvider, fileSplitProvider, partition);
+        return new TreeIndexDropOperatorNodePushable(ctx, storageManager, treeIndexRegistryProvider, fileSplitProvider, partition);
     }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
similarity index 67%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
index c5f3d32..3e34c7b 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -25,27 +25,25 @@
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 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;
 
-public class BTreeDropOperatorNodePushable extends AbstractOperatorNodePushable {
-	private static final Logger LOGGER = Logger.getLogger(BTreeDropOperatorNodePushable.class.getName());
+public class TreeIndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
+	private static final Logger LOGGER = Logger.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
 	
 	private final IHyracksStageletContext ctx;
-    private IIndexRegistryProvider<BTree> btreeRegistryProvider;
+    private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
     private IStorageManagerInterface storageManager;
     private IFileSplitProvider fileSplitProvider;
     private int partition;
 
-    public BTreeDropOperatorNodePushable(IHyracksStageletContext ctx, IStorageManagerInterface storageManager,
-            IIndexRegistryProvider<BTree> btreeRegistryProvider, IFileSplitProvider fileSplitProvider, int partition) {
+    public TreeIndexDropOperatorNodePushable(IHyracksStageletContext ctx, IStorageManagerInterface storageManager,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider, int partition) {
         this.ctx = ctx;
         this.storageManager = storageManager;
-        this.btreeRegistryProvider = btreeRegistryProvider;
+        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
         this.fileSplitProvider = fileSplitProvider;
         this.partition = partition;
     }
@@ -68,7 +66,7 @@
     public void initialize() throws HyracksDataException {
     	try {
 
-    		IndexRegistry<BTree> btreeRegistry = btreeRegistryProvider.getRegistry(ctx);
+    		IndexRegistry<ITreeIndex> treeIndexRegistry = treeIndexRegistryProvider.getRegistry(ctx);
     		IBufferCache bufferCache = storageManager.getBufferCache(ctx);
     		IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
 
@@ -76,27 +74,27 @@
 
     		boolean fileIsMapped = fileMapProvider.isMapped(f);
     		if (!fileIsMapped) {    			    			
-    			throw new HyracksDataException("Cannot drop B-Tree with name " + f.toString() + ". No file mapping exists.");
+    			throw new HyracksDataException("Cannot drop Tree with name " + f.toString() + ". No file mapping exists.");
     		}
 
-    		int btreeFileId = fileMapProvider.lookupFileId(f);
+    		int indexFileId = fileMapProvider.lookupFileId(f);
 
-    		// unregister btree instance
-    		btreeRegistry.lock();
+    		// unregister tree instance
+    		treeIndexRegistry.lock();
     		try {
-    			btreeRegistry.unregister(btreeFileId);
+    			treeIndexRegistry.unregister(indexFileId);
     		} finally {
-    			btreeRegistry.unlock();
+    			treeIndexRegistry.unlock();
     		}
 
     		// remove name to id mapping
-    		bufferCache.deleteFile(btreeFileId);
+    		bufferCache.deleteFile(indexFileId);
     	}
     	// TODO: for the time being we don't throw,
 		// with proper exception handling (no hanging job problem) we should throw
     	catch (Exception e) {
     		if (LOGGER.isLoggable(Level.WARNING)) {
-                LOGGER.warning("BTRee Drop Operator Failed Due To Exception: " + e.getMessage());
+                LOGGER.warning("Tree Drop Operator Failed Due To Exception: " + e.getMessage());
             }
     	}
     }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
similarity index 65%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
index 2694963..2ab24e7 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -25,10 +25,8 @@
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 // re-create in-memory state for a btree that has already been built (i.e., the file exists):
@@ -36,22 +34,22 @@
 // 2. create file mappings (FileMappingProvider)
 // 3. register btree instance (BTreeRegistry)
 
-public class BTreeFileEnlistmentOperatorDescriptor extends AbstractBTreeOperatorDescriptor {
+public class TreeIndexFileEnlistmentOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
-    public BTreeFileEnlistmentOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<BTree> btreeRegistryProvider,
-            IFileSplitProvider fileSplitProvider, IBTreeInteriorFrameFactory interiorFactory,
-            IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories) {
-        super(spec, 0, 0, recDesc, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFactory,
-                leafFactory, typeTraits, comparatorFactories);
+    public TreeIndexFileEnlistmentOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+            IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, 0, 0, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                leafFrameFactory, typeTraits, comparatorFactories, opHelperFactory);
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int partitions) throws HyracksDataException {
-        return new BTreeFileEnlistmentOperatorNodePushable(this, ctx, partition);
+        return new TreeIndexFileEnlistmentOperatorNodePushable(this, ctx, partition);
     }
 
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
similarity index 76%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
index ef78a2d..11c8363 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
@@ -13,22 +13,21 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
 
-public class BTreeFileEnlistmentOperatorNodePushable extends AbstractOperatorNodePushable {
+public class TreeIndexFileEnlistmentOperatorNodePushable extends AbstractOperatorNodePushable {
 
-    private final BTreeOpHelper btreeOpHelper;
+    private final TreeIndexOpHelper btreeOpHelper;
 
-    public BTreeFileEnlistmentOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+    public TreeIndexFileEnlistmentOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
             int partition) {
-        btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.ENLIST);
+        btreeOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.ENLIST);
     }
 
     @Override
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
similarity index 63%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 264bf12..a1af38c 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -13,7 +13,7 @@
  * limitations under the License.
  */
 
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -24,28 +24,26 @@
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
-public class BTreeInsertUpdateDeleteOperatorDescriptor extends AbstractBTreeOperatorDescriptor {
+public class TreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     private final int[] fieldPermutation;
 
-    private TreeIndexOp op;
+    private IndexOp op;
 
-    public BTreeInsertUpdateDeleteOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<BTree> btreeRegistryProvider,
-            IFileSplitProvider fileSplitProvider, IBTreeInteriorFrameFactory interiorFactory,
-            IBTreeLeafFrameFactory leafFactory, ITypeTrait[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, TreeIndexOp op) {
-        super(spec, 1, 1, recDesc, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFactory,
-                leafFactory, typeTraits, comparatorFactories);
+    public TreeIndexInsertUpdateDeleteOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+            IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> btreeRegistryProvider,
+            IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, IndexOp op, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, 1, 1, recDesc, storageManager, btreeRegistryProvider, fileSplitProvider, interiorFrameFactory,
+                leafFrameFactory, typeTraits, comparatorFactories, opHelperFactory);
         this.fieldPermutation = fieldPermutation;
         this.op = op;
     }
@@ -53,7 +51,7 @@
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        return new BTreeInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
+        return new TreeIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
                 recordDescProvider, op);
     }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
similarity index 64%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index a8d6725..d41848b 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.nio.ByteBuffer;
 
@@ -23,25 +23,24 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOpContext;
 
-public class BTreeInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private final BTreeOpHelper btreeOpHelper;
+public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    private final TreeIndexOpHelper treeIndexOpHelper;
     private FrameTupleAccessor accessor;
     private final IRecordDescriptorProvider recordDescProvider;
-    private final TreeIndexOp op;
+    private final IndexOp op;
     private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
     private ByteBuffer writeBuffer;
-    private BTreeOpContext opCtx;
+    private IndexOpContext opCtx;
 
-    public BTreeInsertUpdateDeleteOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc,
+    public TreeIndexInsertUpdateDeleteOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
             IHyracksStageletContext ctx, int partition, int[] fieldPermutation,
-            IRecordDescriptorProvider recordDescProvider, TreeIndexOp op) {
-        btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
+            IRecordDescriptorProvider recordDescProvider, IndexOp op) {
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
         this.recordDescProvider = recordDescProvider;
         this.op = op;
         tuple.setFieldPermutation(fieldPermutation);
@@ -49,25 +48,25 @@
 
     @Override
     public void open() throws HyracksDataException {
-        AbstractBTreeOperatorDescriptor opDesc = (AbstractBTreeOperatorDescriptor)btreeOpHelper.getOperatorDescriptor();
+        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor)treeIndexOpHelper.getOperatorDescriptor();
     	RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-    	accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
-    	writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
+    	accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
+    	writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
     	try {
-    		btreeOpHelper.init();
-    		btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
-    		opCtx = btreeOpHelper.getBTree().createOpContext(op, btreeOpHelper.getLeafFrame(),
-    				btreeOpHelper.getInteriorFrame(), new LIFOMetaDataFrame());
+    		treeIndexOpHelper.init();
+    		treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
+    		opCtx = treeIndexOpHelper.getTreeIndex().createOpContext(op, treeIndexOpHelper.getLeafFrame(),
+    				treeIndexOpHelper.getInteriorFrame(), new LIFOMetaDataFrame());
     	} catch(Exception e) {
     		// cleanup in case of failure
-    		btreeOpHelper.deinit();
+    		treeIndexOpHelper.deinit();
     		throw new HyracksDataException(e);
     	}
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        final BTree btree = btreeOpHelper.getBTree();
+        final ITreeIndex treeIndex = treeIndexOpHelper.getTreeIndex();
         accessor.reset(buffer);
 
         int tupleCount = accessor.getTupleCount();
@@ -76,13 +75,13 @@
             try {
                 switch (op) {
 
-                    case TI_INSERT: {
-                        btree.insert(tuple, opCtx);
+                    case INSERT: {
+                        treeIndex.insert(tuple, opCtx);
                     }
                         break;
 
-                    case TI_DELETE: {
-                        btree.delete(tuple, opCtx);
+                    case DELETE: {
+                        treeIndex.delete(tuple, opCtx);
                     }
                         break;
 
@@ -108,7 +107,7 @@
         try {
             writer.close();
         } finally {
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
         }
     }
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
new file mode 100644
index 0000000..a48bec4
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
@@ -0,0 +1,179 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class TreeIndexOpHelper {   
+    
+	protected ITreeIndexFrame interiorFrame;
+	protected ITreeIndexFrame leafFrame;
+	protected MultiComparator cmp;
+	
+	protected ITreeIndex treeIndex;
+	protected int indexFileId = -1;
+	protected int partition;
+
+	protected ITreeIndexOperatorDescriptorHelper opDesc;
+	protected IHyracksStageletContext ctx;
+
+	protected IndexHelperOpenMode mode;
+
+    public TreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx, int partition,
+            IndexHelperOpenMode mode) {
+        this.opDesc = opDesc;
+        this.ctx = ctx;
+        this.mode = mode;
+        this.partition = partition;
+    }
+
+	public void init() throws HyracksDataException {
+        IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+        IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
+        IFileSplitProvider fileSplitProvider = opDesc.getTreeIndexFileSplitProvider();
+
+        FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
+        boolean fileIsMapped = fileMapProvider.isMapped(f);
+
+		switch (mode) {
+		
+		case OPEN: {
+			if (!fileIsMapped) {
+				throw new HyracksDataException(
+						"Trying to open btree from unmapped file " + f.toString());
+			}
+		}
+		break;
+
+		case CREATE:
+		case ENLIST: {
+			if (!fileIsMapped) {
+				bufferCache.createFile(f);
+			}
+		}
+		break;
+		
+		}
+		
+        int fileId = fileMapProvider.lookupFileId(f);		
+        try {
+        	bufferCache.openFile(fileId);
+        } catch(HyracksDataException e) {
+        	// revert state of buffer cache since file failed to open
+        	if(!fileIsMapped) {
+        		bufferCache.deleteFile(fileId);
+        	}
+        	throw e;
+        }
+        
+        // only set indexFileId member when openFile() succeeds, 
+        // otherwise deinit() will try to close the file that failed to open
+        indexFileId = fileId;
+
+		interiorFrame = opDesc.getTreeIndexInteriorFactory().createFrame();
+		leafFrame = opDesc.getTreeIndexLeafFactory().createFrame();
+
+        IndexRegistry<ITreeIndex> treeIndexRegistry = opDesc.getTreeIndexRegistryProvider().getRegistry(ctx);
+        treeIndex = treeIndexRegistry.get(indexFileId);
+        if (treeIndex == null) {
+
+			// create new tree and register it
+            treeIndexRegistry.lock();
+			try {
+				// check if tree has already been registered by another thread
+			    treeIndex = treeIndexRegistry.get(indexFileId);
+				if (treeIndex == null) {
+					// this thread should create and register the tree
+
+					IBinaryComparator[] comparators = new IBinaryComparator[opDesc
+							.getTreeIndexComparatorFactories().length];
+					for (int i = 0; i < opDesc.getTreeIndexComparatorFactories().length; i++) {
+						comparators[i] = opDesc.getTreeIndexComparatorFactories()[i]
+								.createBinaryComparator();
+					}
+
+					cmp = new MultiComparator(opDesc
+							.getTreeIndexTypeTraits(), comparators);
+					
+					treeIndex = createTreeIndex();
+					if (mode == IndexHelperOpenMode.CREATE) {
+						ITreeIndexMetaDataFrame metaFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+						try {
+						    treeIndex.create(indexFileId, leafFrame, metaFrame);							
+						} catch (Exception e) {
+							throw new HyracksDataException(e);
+						}
+					}
+					treeIndex.open(indexFileId);
+					treeIndexRegistry.register(indexFileId, treeIndex);
+				}
+			} finally {
+			    treeIndexRegistry.unlock();
+			}
+		}
+	}
+
+	// MUST be overridden
+	public ITreeIndex createTreeIndex() throws HyracksDataException {
+		throw new HyracksDataException("createTreeIndex Operation not implemented.");
+	}
+	
+	// MUST be overridden
+	public ITreeIndexCursor createDiskOrderScanCursor(ITreeIndexFrame leafFrame) throws HyracksDataException {
+		throw new HyracksDataException("createDiskOrderScanCursor Operation not implemented.");
+	}
+	
+    public void deinit() throws HyracksDataException {
+        if (indexFileId != -1) {
+            IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+            bufferCache.closeFile(indexFileId);
+        }
+    }
+
+	public ITreeIndex getTreeIndex() {
+		return treeIndex;
+	}
+
+    public IHyracksStageletContext getHyracksStageletContext() {
+        return ctx;
+    }
+
+	public ITreeIndexOperatorDescriptorHelper getOperatorDescriptor() {
+		return opDesc;
+	}
+
+	public ITreeIndexFrame getLeafFrame() {
+		return leafFrame;
+	}
+
+	public ITreeIndexFrame getInteriorFrame() {
+		return interiorFrame;
+	}
+
+	public int getIndexFileId() {
+		return indexFileId;
+	}
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
new file mode 100644
index 0000000..1c0c734
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class TreeIndexStatsOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    
+    public TreeIndexStatsOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+            IBinaryComparatorFactory[] comparatorFactories, ITreeIndexOpHelperFactory opHelperFactory) {
+        super(spec, 0, 0, null, storageManager, treeIndexRegistryProvider, fileSplitProvider, interiorFrameFactory, leafFrameFactory,
+                typeTraits, comparatorFactories, opHelperFactory);
+    }
+    
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new TreeIndexStatsOperatorNodePushable(this, ctx, partition);
+    }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeStatsOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
similarity index 63%
rename from hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeStatsOperatorNodePushable.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index 8fc77e83..11ac96f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeStatsOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -12,29 +12,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.utility.TreeIndexStats;
 import edu.uci.ics.hyracks.storage.am.common.utility.TreeIndexStatsGatherer;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
-public class BTreeStatsOperatorNodePushable extends
+public class TreeIndexStatsOperatorNodePushable extends
 AbstractOperatorNodePushable {
-	private final BTreeOpHelper btreeOpHelper;
+	private final TreeIndexOpHelper treeIndexOpHelper;
 	private final IHyracksStageletContext ctx;
 	private TreeIndexStatsGatherer statsGatherer;
 
-	public BTreeStatsOperatorNodePushable(
-			AbstractBTreeOperatorDescriptor opDesc,
+	public TreeIndexStatsOperatorNodePushable(
+			AbstractTreeIndexOperatorDescriptor opDesc,
 			IHyracksStageletContext ctx, int partition) {
-		btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition,
+		treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
 		        IndexHelperOpenMode.CREATE);
 		this.ctx = ctx;
 	}
@@ -56,17 +55,17 @@
 	@Override
 	public void initialize() throws HyracksDataException {
 		try {
-			btreeOpHelper.init();
-			btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
+			treeIndexOpHelper.init();
+			treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
 
-			BTree btree = btreeOpHelper.getBTree();			
-			IBufferCache bufferCache = btreeOpHelper.getOperatorDescriptor().getStorageManager().getBufferCache(ctx);
+			ITreeIndex treeIndex = treeIndexOpHelper.getTreeIndex();			
+			IBufferCache bufferCache = treeIndexOpHelper.getOperatorDescriptor().getStorageManager().getBufferCache(ctx);
 
-			statsGatherer = new TreeIndexStatsGatherer(bufferCache, btree.getFreePageManager(), btreeOpHelper.getBTreeFileId(), btree.getRootPageId());
-			TreeIndexStats stats = statsGatherer.gatherStats(btree.getLeafFrameFactory().getFrame(), btree.getInteriorFrameFactory().getFrame(), btree.getFreePageManager().getMetaDataFrameFactory().getFrame());
+			statsGatherer = new TreeIndexStatsGatherer(bufferCache, treeIndex.getFreePageManager(), treeIndexOpHelper.getIndexFileId(), treeIndex.getRootPageId());
+			TreeIndexStats stats = statsGatherer.gatherStats(treeIndex.getLeafFrameFactory().createFrame(), treeIndex.getInteriorFrameFactory().createFrame(), treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame());
 			System.err.println(stats.toString());
 		} catch (Exception e) {
-			btreeOpHelper.deinit();
+			treeIndexOpHelper.deinit();
 			throw new HyracksDataException(e);
 		}
 	}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrameFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrameFactory.java
index eb56987..409c8b2 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrameFactory.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/frames/LIFOMetaDataFrameFactory.java
@@ -20,7 +20,7 @@
 
 public class LIFOMetaDataFrameFactory implements ITreeIndexMetaDataFrameFactory {
     @Override
-    public ITreeIndexMetaDataFrame getFrame() {
+    public ITreeIndexMetaDataFrame createFrame() {
         return new LIFOMetaDataFrame();
     }
 }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/TreeIndexOp.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOp.java
similarity index 90%
rename from hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/TreeIndexOp.java
rename to hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOp.java
index 549c161..61474f7 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/TreeIndexOp.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOp.java
@@ -15,6 +15,6 @@
 
 package edu.uci.ics.hyracks.storage.am.common.ophelpers;
 
-public enum TreeIndexOp {
-    TI_INSERT, TI_DELETE, TI_UPDATE, TI_SEARCH
+public enum IndexOp {
+    INSERT, DELETE, UPDATE, SEARCH
 }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOpContext.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOpContext.java
new file mode 100644
index 0000000..9122174
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/IndexOpContext.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.hyracks.storage.am.common.ophelpers;
+
+public interface IndexOpContext {
+	void reset();
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
index d89e4c2..8cc0bd7 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
@@ -21,10 +21,10 @@
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
@@ -38,12 +38,13 @@
 
     // btree
     protected final IFileSplitProvider btreeFileSplitProvider;
-    protected final IIndexRegistryProvider<BTree> btreeRegistryProvider;
-    protected final IBTreeInteriorFrameFactory interiorFrameFactory;
-    protected final IBTreeLeafFrameFactory leafFrameFactory;
+    protected final IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
+    protected final ITreeIndexFrameFactory interiorFrameFactory;
+    protected final ITreeIndexFrameFactory leafFrameFactory;
     protected final ITypeTrait[] btreeTypeTraits;
     protected final IBinaryComparatorFactory[] btreeComparatorFactories;
-
+    protected final ITreeIndexOpHelperFactory opHelperFactory;
+    
     // inverted index
     protected final IFileSplitProvider invIndexFileSplitProvider;
     protected final IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider;    
@@ -52,9 +53,10 @@
 
     public AbstractInvertedIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
             RecordDescriptor recDesc, IStorageManagerInterface storageManager,
-            IFileSplitProvider btreeFileSplitProvider, IIndexRegistryProvider<BTree> btreeRegistryProvider,
-            IBTreeInteriorFrameFactory interiorFrameFactory, IBTreeLeafFrameFactory leafFrameFactory,
+            IFileSplitProvider btreeFileSplitProvider, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+            ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
             ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
+            ITreeIndexOpHelperFactory opHelperFactory,
             IFileSplitProvider invIndexFileSplitProvider,
             IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
             IBinaryComparatorFactory[] invIndexComparatorFactories) {
@@ -65,11 +67,12 @@
 
         // btree
         this.btreeFileSplitProvider = btreeFileSplitProvider;
-        this.btreeRegistryProvider = btreeRegistryProvider;
+        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
         this.interiorFrameFactory = interiorFrameFactory;
         this.leafFrameFactory = leafFrameFactory;
         this.btreeTypeTraits = btreeTypeTraits;
         this.btreeComparatorFactories = btreeComparatorFactories;
+        this.opHelperFactory = opHelperFactory;
 
         // inverted index
         this.invIndexFileSplitProvider = invIndexFileSplitProvider;
@@ -82,27 +85,27 @@
     }
 
     @Override
-    public IFileSplitProvider getBTreeFileSplitProvider() {
+    public IFileSplitProvider getTreeIndexFileSplitProvider() {
         return btreeFileSplitProvider;
     }
 
     @Override
-    public IBinaryComparatorFactory[] getBTreeComparatorFactories() {
+    public IBinaryComparatorFactory[] getTreeIndexComparatorFactories() {
         return btreeComparatorFactories;
     }
 
     @Override
-    public ITypeTrait[] getBTreeTypeTraits() {
+    public ITypeTrait[] getTreeIndexTypeTraits() {
         return btreeTypeTraits;
     }
 
     @Override
-    public IBTreeInteriorFrameFactory getBTreeInteriorFactory() {
+    public ITreeIndexFrameFactory getTreeIndexInteriorFactory() {
         return interiorFrameFactory;
     }
 
     @Override
-    public IBTreeLeafFrameFactory getBTreeLeafFactory() {
+    public ITreeIndexFrameFactory getTreeIndexLeafFactory() {
         return leafFrameFactory;
     }
 
@@ -112,8 +115,8 @@
     }
 
     @Override
-    public IIndexRegistryProvider<BTree> getBTreeRegistryProvider() {
-        return btreeRegistryProvider;
+    public IIndexRegistryProvider<ITreeIndex> getTreeIndexRegistryProvider() {
+        return treeIndexRegistryProvider;
     }
 
     @Override
@@ -140,4 +143,9 @@
     public ITypeTrait[] getInvIndexTypeTraits() {
         return invIndexTypeTraits;
     }
+    
+    @Override
+    public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory() {
+        return opHelperFactory;
+    }
 }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/IInvertedIndexOperatorDescriptorHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/IInvertedIndexOperatorDescriptorHelper.java
index c7024a1..762a86c 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/IInvertedIndexOperatorDescriptorHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/IInvertedIndexOperatorDescriptorHelper.java
@@ -3,11 +3,11 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeOperatorDescriptorHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 
-public interface IInvertedIndexOperatorDescriptorHelper extends IBTreeOperatorDescriptorHelper {            
+public interface IInvertedIndexOperatorDescriptorHelper extends ITreeIndexOperatorDescriptorHelper {            
     public IFileSplitProvider getInvIndexFileSplitProvider();
 
     public IBinaryComparatorFactory[] getInvIndexComparatorFactories();
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index 439ca94..2aa0e0a 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -23,10 +23,10 @@
 import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -41,15 +41,16 @@
 
     public InvertedIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
             int[] fieldPermutation, IFileSplitProvider btreeFileSplitProvider,
-            IIndexRegistryProvider<BTree> btreeRegistryProvider, IBTreeInteriorFrameFactory interiorFrameFactory,
-            IBTreeLeafFrameFactory leafFrameFactory, ITypeTrait[] btreeTypeTraits,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] btreeTypeTraits,
             IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
+            ITreeIndexOpHelperFactory opHelperFactory,
             IFileSplitProvider invIndexFileSplitProvider,
             IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
             IBinaryComparatorFactory[] invIndexComparatorFactories, IInvertedListBuilder invListBuilder) {
-        super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, btreeRegistryProvider, interiorFrameFactory,
+        super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, treeIndexRegistryProvider, interiorFrameFactory,
                 leafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeFillFactor,
-                invIndexFileSplitProvider, invIndexRegistryProvider, invIndexTypeTraits, invIndexComparatorFactories);
+                opHelperFactory, invIndexFileSplitProvider, invIndexRegistryProvider, invIndexTypeTraits, invIndexComparatorFactories);
         this.fieldPermutation = fieldPermutation;
         this.btreeFillFactor = btreeFillFactor;
         this.invListBuilder = invListBuilder;
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
index 21933df..060f166 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
@@ -22,14 +22,14 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelper;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.PermutingFrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 
 public class InvertedIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private final BTreeOpHelper btreeOpHelper;
+    private final TreeIndexOpHelper treeIndexOpHelper;
     private float btreeFillFactor;
     
     private final InvertedIndexOpHelper invIndexOpHelper;        
@@ -46,7 +46,7 @@
     public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
             IHyracksStageletContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
             IInvertedListBuilder invListBuilder, IRecordDescriptorProvider recordDescProvider) {
-        btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
         invIndexOpHelper = new InvertedIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
         this.btreeFillFactor = btreeFillFactor;
         this.recordDescProvider = recordDescProvider;
@@ -57,18 +57,18 @@
 
     @Override
     public void open() throws HyracksDataException {
-        AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) btreeOpHelper
+        AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) treeIndexOpHelper
                 .getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
         
         // btree
         try {
-            btreeOpHelper.init();
-            btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());            
+            treeIndexOpHelper.init();
+            treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());            
         } catch (Exception e) {
             // cleanup in case of failure
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
             throw new HyracksDataException(e);
         }
         
@@ -99,7 +99,7 @@
         try {
             invIndexOpHelper.getInvIndex().endBulkLoad(bulkLoadCtx);
         } finally {
-            btreeOpHelper.deinit();
+            treeIndexOpHelper.deinit();
         }
     }
 
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
index 025462d..4beefcd 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
@@ -19,8 +19,9 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
@@ -112,15 +113,15 @@
 							.getInvIndexTypeTraits(), comparators);
 					
 					// assumes btree has already been registered
-					IFileSplitProvider btreeFileSplitProvider = opDesc.getBTreeFileSplitProvider();
-					IndexRegistry<BTree> btreeRegistry = opDesc.getBTreeRegistryProvider().getRegistry(ctx);
+					IFileSplitProvider btreeFileSplitProvider = opDesc.getTreeIndexFileSplitProvider();
+					IndexRegistry<ITreeIndex> treeIndexRegistry = opDesc.getTreeIndexRegistryProvider().getRegistry(ctx);
 					FileReference btreeFile = btreeFileSplitProvider.getFileSplits()[partition].getLocalFile();
 					boolean btreeFileIsMapped = fileMapProvider.isMapped(btreeFile);
 					if(!btreeFileIsMapped) {
 					    throw new HyracksDataException("Trying to create inverted index, but associated BTree file has not been mapped");
 					}
 					int btreeFileId = fileMapProvider.lookupFileId(f);					
-					BTree btree = btreeRegistry.get(btreeFileId);					
+					BTree btree = (BTree)treeIndexRegistry.get(btreeFileId);					
 					
 					invIndex = new InvertedIndex(bufferCache, btree, cmp);					
 					invIndex.open(invIndexFileId);
@@ -147,7 +148,7 @@
         return ctx;
     }
 
-	public IBTreeOperatorDescriptorHelper getOperatorDescriptor() {
+	public ITreeIndexOperatorDescriptorHelper getOperatorDescriptor() {
 		return opDesc;
 	}
 	
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 21a5e4c..fc3cf15 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -12,10 +12,11 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
@@ -129,7 +130,7 @@
         }
     }    
     
-    public boolean openCursor(IBTreeCursor btreeCursor, RangePredicate btreePred, BTreeOpContext btreeOpCtx, IInvertedListCursor invListCursor) throws Exception {
+    public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, BTreeOpContext btreeOpCtx, IInvertedListCursor invListCursor) throws Exception {
         btree.search(btreeCursor, btreePred, btreeOpCtx);       
         
         boolean ret = false;
@@ -208,7 +209,7 @@
         private final FrameTupleAccessor btreeFrameTupleAccessor;
         private final FrameTupleReference btreeFrameTupleReference = new FrameTupleReference();
         private final float btreeFillFactor;
-        private BTree.BulkLoadContext btreeBulkLoadCtx;
+        private IIndexBulkLoadContext btreeBulkLoadCtx;
 
         private int currentInvListStartPageId;
         private int currentInvListStartOffset;
@@ -237,9 +238,9 @@
         }
 
         public void init(int startPageId, int fileId) throws HyracksDataException {
-            btreeBulkLoadCtx = btree.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR, btree.getLeafFrameFactory().getFrame(),
-                    btree.getInteriorFrameFactory().getFrame(), btree.getFreePageManager().getMetaDataFrameFactory()
-                            .getFrame());
+            btreeBulkLoadCtx = btree.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR, btree.getLeafFrameFactory().createFrame(),
+                    btree.getInteriorFrameFactory().createFrame(), btree.getFreePageManager().getMetaDataFrameFactory()
+                            .createFrame());
             currentPageId = startPageId;
             currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
             currentPage.acquireWriteLatch();
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index b2f289a..41d831f 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -37,14 +37,14 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 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.api.IBTreeCursor;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexResultCursor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearcher;
@@ -64,9 +64,9 @@
 	protected List<ByteBuffer> swap = null;
 	protected int maxResultBufIdx = 0;
 
-	protected final IBTreeLeafFrame leafFrame;
-	protected final IBTreeInteriorFrame interiorFrame;
-	protected final IBTreeCursor btreeCursor;
+	protected final ITreeIndexFrame leafFrame;
+	protected final ITreeIndexFrame interiorFrame;
+	protected final ITreeIndexCursor btreeCursor;
 	protected final FrameTupleReference searchKey = new FrameTupleReference();
 	protected final RangePredicate btreePred = new RangePredicate(true, null, null, true, true, null, null);
 	protected final BTreeOpContext btreeOpCtx;
@@ -92,10 +92,10 @@
 		this.invIndex = invIndex;
 		this.queryTokenizer = queryTokenizer;
 
-		leafFrame = invIndex.getBTree().getLeafFrameFactory().getFrame();
-		interiorFrame = invIndex.getBTree().getInteriorFrameFactory().getFrame();
+		leafFrame = invIndex.getBTree().getLeafFrameFactory().createFrame();
+		interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
 
-		btreeCursor = new RangeSearchCursor(leafFrame);
+		btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame)leafFrame);
 		ITypeTrait[] invListFields = invIndex.getInvListElementCmp().getTypeTraits();
 		invListFieldsWithCount = new TypeTrait[invListFields.length + 1];
 		int tmp = 0;
@@ -107,7 +107,7 @@
 		invListFieldsWithCount[invListFields.length] = new TypeTrait(4);
 		invListKeyLength = tmp;
 
-		btreeOpCtx = invIndex.getBTree().createOpContext(TreeIndexOp.TI_SEARCH, leafFrame,
+		btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH, leafFrame,
 				interiorFrame, null);
 
 		resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index 1c9c32a..ed70d1b 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -13,7 +13,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeCursor;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrameFactory;
@@ -215,7 +215,7 @@
         fileId = -1;
     }
 
-    public RTreeOpContext createOpContext(TreeIndexOp op, IRTreeFrame interiorFrame, IRTreeFrame leafFrame,
+    public RTreeOpContext createOpContext(IndexOp op, IRTreeFrame interiorFrame, IRTreeFrame leafFrame,
             ITreeIndexMetaDataFrame metaFrame, String threadName) {
         // TODO: figure out better tree-height hint
         return new RTreeOpContext(op, interiorFrame, leafFrame, metaFrame, 8, interiorCmp.getKeyFieldCount() / 2,
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
index 91a2e24..976c45e 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
@@ -2,12 +2,12 @@
 
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeCursor;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
 
 public final class RTreeOpContext {
-    public final TreeIndexOp op;
+    public final IndexOp op;
     public final IRTreeFrame interiorFrame;
     public final IRTreeFrame leafFrame;
     public IRTreeCursor cursor;
@@ -22,7 +22,7 @@
     public Rectangle[] rec;
     public String threadName; // for debugging
 
-    public RTreeOpContext(TreeIndexOp op, IRTreeFrame interiorFrame, IRTreeFrame leafFrame,
+    public RTreeOpContext(IndexOp op, IRTreeFrame interiorFrame, IRTreeFrame leafFrame,
             ITreeIndexMetaDataFrame metaFrame, int treeHeightHint, int dim, String threadName) {
         this.op = op;
         this.interiorFrame = interiorFrame;
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index b9e7ec4..2325db7 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -15,7 +15,7 @@
 package edu.uci.ics.hyracks.test.support;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -30,7 +30,7 @@
 public class TestStorageManagerComponentHolder {
     private static IBufferCache bufferCache;
     private static IFileMapProvider fileMapProvider;
-    private static IndexRegistry<BTree> btreeRegistry;
+    private static IndexRegistry<ITreeIndex> treeIndexRegistry;
 
     private static int pageSize;
     private static int numPages;
@@ -42,7 +42,7 @@
         TestStorageManagerComponentHolder.maxOpenFiles = maxOpenFiles;
         bufferCache = null;
         fileMapProvider = null;
-        btreeRegistry = null;
+        treeIndexRegistry = null;
     }
 
     public synchronized static IBufferCache getBufferCache(IHyracksStageletContext ctx) {
@@ -63,10 +63,10 @@
         return fileMapProvider;
     }
 
-    public synchronized static IndexRegistry<BTree> getBTreeRegistry(IHyracksStageletContext ctx) {
-        if (btreeRegistry == null) {
-            btreeRegistry = new IndexRegistry<BTree>();
+    public synchronized static IndexRegistry<ITreeIndex> getTreeIndexRegistry(IHyracksStageletContext ctx) {
+        if (treeIndexRegistry == null) {
+        	treeIndexRegistry = new IndexRegistry<ITreeIndex>();
         }
-        return btreeRegistry;
+        return treeIndexRegistry;
     }
 }
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
similarity index 76%
rename from hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
rename to hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
index e1c0feb..630f47f 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
@@ -15,15 +15,15 @@
 package edu.uci.ics.hyracks.test.support;
 
 import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 
-public class TestBTreeRegistryProvider implements IIndexRegistryProvider<BTree> {
+public class TestTreeIndexRegistryProvider implements IIndexRegistryProvider<ITreeIndex> {
     private static final long serialVersionUID = 1L;
     
 	@Override
-	public IndexRegistry<BTree> getRegistry(IHyracksStageletContext ctx) {
-		return TestStorageManagerComponentHolder.getBTreeRegistry(ctx);
+	public IndexRegistry<ITreeIndex> getRegistry(IHyracksStageletContext ctx) {
+		return TestStorageManagerComponentHolder.getTreeIndexRegistry(ctx);
 	}
 }
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index df7be4f..9fad24c 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -22,21 +22,20 @@
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.common.utility.TreeIndexBufferCacheWarmup;
 import edu.uci.ics.hyracks.storage.am.common.utility.TreeIndexStats;
@@ -86,15 +85,15 @@
 
 		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
 				typeTraits);
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+		IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(
 				bufferCache, fileId, 0, metaFrameFactory);
@@ -126,7 +125,7 @@
 		FrameTupleReference tuple = new FrameTupleReference();
 
 		BTreeOpContext insertOpCtx = btree.createOpContext(
-				TreeIndexOp.TI_INSERT, leafFrame, interiorFrame, metaFrame);
+				IndexOp.INSERT, leafFrame, interiorFrame, metaFrame);
 
 		// 10000
 		for (int i = 0; i < 100000; i++) {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
index 14b5903..e8c8065 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
@@ -39,26 +39,27 @@
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 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.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.DiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeDiskOrderScanCursor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -110,15 +111,15 @@
 
 		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
 				typeTraits);		
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+		IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 		
@@ -148,7 +149,7 @@
 		accessor.reset(frame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 
 		// 10000
@@ -204,10 +205,10 @@
 		// ordered scan
 
 		print("ORDERED SCAN:\n");
-		IBTreeCursor scanCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
 		RangePredicate nullPred = new RangePredicate(true, null, null, true,
 				true, null, null);
-		BTreeOpContext searchOpCtx = btree.createOpContext(TreeIndexOp.TI_SEARCH,
+		BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH,
 				leafFrame, interiorFrame, null);
 		btree.search(scanCursor, nullPred, searchOpCtx);
 		try {
@@ -225,7 +226,7 @@
 
 		// disk-order scan
 		print("DISK-ORDER SCAN:\n");
-		DiskOrderScanCursor diskOrderCursor = new DiskOrderScanCursor(leafFrame);
+		BTreeDiskOrderScanCursor diskOrderCursor = new BTreeDiskOrderScanCursor(leafFrame);
 		btree.diskOrderScan(diskOrderCursor, leafFrame, metaFrame);
 		try {
 			while (diskOrderCursor.hasNext()) {
@@ -243,7 +244,7 @@
 		// range search in [-1000, 1000]
 		print("RANGE SEARCH:\n");
 
-		IBTreeCursor rangeCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
 		// build low and high keys
 		ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -348,15 +349,15 @@
 				typeTraits);
 		// SimpleTupleWriterFactory tupleWriterFactory = new
 		// SimpleTupleWriterFactory();
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 		
@@ -387,7 +388,7 @@
 		accessor.reset(frame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 
 		for (int i = 0; i < 10000; i++) {
@@ -426,10 +427,10 @@
 
 		// try a simple index scan
 		print("ORDERED SCAN:\n");
-		IBTreeCursor scanCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
 		RangePredicate nullPred = new RangePredicate(true, null, null, true,
 				true, null, null);
-		BTreeOpContext searchOpCtx = btree.createOpContext(TreeIndexOp.TI_SEARCH,
+		BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH,
 				leafFrame, interiorFrame, null);
 		btree.search(scanCursor, nullPred, searchOpCtx);
 
@@ -448,7 +449,7 @@
 
 		// range search in [(-3),(3)]
 		print("RANGE SEARCH:\n");
-		IBTreeCursor rangeCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
 		// build low and high keys
 		ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -555,15 +556,15 @@
 		SimpleTupleWriterFactory tupleWriterFactory = new SimpleTupleWriterFactory();
 		// TypeAwareTupleWriterFactory tupleWriterFactory = new
 		// TypeAwareTupleWriterFactory(typeTraits);
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 		
@@ -589,7 +590,7 @@
 		accessor.reset(frame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 		int maxLength = 10; // max string length to be generated
 		for (int i = 0; i < 10000; i++) {
@@ -629,10 +630,10 @@
 
 		// ordered scan
 		print("ORDERED SCAN:\n");
-		IBTreeCursor scanCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
 		RangePredicate nullPred = new RangePredicate(true, null, null, true,
 				true, null, null);
-		BTreeOpContext searchOpCtx = btree.createOpContext(TreeIndexOp.TI_SEARCH,
+		BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH,
 				leafFrame, interiorFrame, null);
 		btree.search(scanCursor, nullPred, searchOpCtx);
 
@@ -652,7 +653,7 @@
 		// range search in ["cbf", cc7"]
 		print("RANGE SEARCH:\n");
 
-		IBTreeCursor rangeCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
 		// build low and high keys
 		ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
@@ -755,15 +756,15 @@
 		// SimpleTupleWriterFactory();
 		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
 				typeTraits);
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 		
@@ -789,9 +790,9 @@
 		accessor.reset(frame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
-		BTreeOpContext deleteOpCtx = btree.createOpContext(TreeIndexOp.TI_DELETE,
+		BTreeOpContext deleteOpCtx = btree.createOpContext(IndexOp.DELETE,
 				leafFrame, interiorFrame, metaFrame);
 
 		int runs = 3;
@@ -942,15 +943,15 @@
 		// SimpleTupleWriterFactory();
 		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
 				typeTraits);
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+		ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 		
@@ -977,7 +978,7 @@
 		accessor.reset(frame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		BTree.BulkLoadContext bulkLoadCtx = btree.beginBulkLoad(0.7f,
+		IIndexBulkLoadContext bulkLoadCtx = btree.beginBulkLoad(0.7f,
 				leafFrame, interiorFrame, metaFrame);
 
 		// generate sorted records
@@ -1013,7 +1014,7 @@
 
 		// range search
 		print("RANGE SEARCH:\n");
-		IBTreeCursor rangeCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame)leafFrame);
 
 		// build low and high keys
 		ArrayTupleBuilder ktb = new ArrayTupleBuilder(1);
@@ -1056,7 +1057,7 @@
 		// TODO: check when searching backwards
 		RangePredicate rangePred = new RangePredicate(true, lowKey, highKey,
 				true, true, searchCmp, searchCmp);
-		BTreeOpContext searchOpCtx = btree.createOpContext(TreeIndexOp.TI_SEARCH,
+		BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH,
 				leafFrame, interiorFrame, null);
 		btree.search(rangeCursor, rangePred, searchOpCtx);
 
@@ -1118,15 +1119,15 @@
 		// SimpleTupleWriterFactory();
 		TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
 				typeTraits);
-		IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+		ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 				tupleWriterFactory);
 		ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-		IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+        IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+		ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 		IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
 		
@@ -1188,7 +1189,7 @@
 		intervals[9][0] = 20;
 		intervals[9][1] = 35;
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 
 		// int exceptionCount = 0;
@@ -1230,10 +1231,10 @@
 		// try a simple index scan
 
 		print("ORDERED SCAN:\n");
-		IBTreeCursor scanCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
 		RangePredicate nullPred = new RangePredicate(true, null, null, true,
 				true, null, null);
-		BTreeOpContext searchOpCtx = btree.createOpContext(TreeIndexOp.TI_SEARCH,
+		BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH,
 				leafFrame, interiorFrame, null);
 		btree.search(scanCursor, nullPred, searchOpCtx);
 
@@ -1252,7 +1253,7 @@
 
 		// try a range search
 		print("RANGE SEARCH:\n");
-		IBTreeCursor rangeCursor = new RangeSearchCursor(leafFrame);
+		ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
 		// build low and high keys
 		ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 9ee3cb6..bc6f991 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -45,11 +45,8 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeCursor;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.FieldPrefixNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
@@ -57,14 +54,16 @@
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeException;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -83,15 +82,15 @@
 
 	TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
 			typeTraits);
-	IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
+	ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(
 			tupleWriterFactory);
-	IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
+	ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(
 			tupleWriterFactory);
 	ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-	IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-	IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-	ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+	IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+	IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
+	ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
 	IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
 	ByteBuffer frame = ctx.allocateFrame();
@@ -147,7 +146,7 @@
 		ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
 		DataOutput dos = tb.getDataOutput();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 
 		// generate keys
@@ -254,7 +253,7 @@
 		ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
 		DataOutput dos = tb.getDataOutput();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 
 		// generate keys
@@ -329,9 +328,9 @@
 		System.out
 				.println("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
 
-		IBTreeLeafFrameFactory leafFrameFactory = new FieldPrefixNSMLeafFrameFactory(
+		ITreeIndexFrameFactory leafFrameFactory = new FieldPrefixNSMLeafFrameFactory(
 				tupleWriterFactory);
-		IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
+		IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
 
 		TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
 		IBufferCache bufferCache = TestStorageManagerComponentHolder
@@ -363,7 +362,7 @@
 		ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
 		DataOutput dos = tb.getDataOutput();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(TreeIndexOp.TI_INSERT,
+		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT,
 				leafFrame, interiorFrame, metaFrame);
 
 		// generate keys
@@ -539,13 +538,13 @@
 				int lowKey = i;
 				int highKey = j;
 
-				IBTreeCursor rangeCursor = new RangeSearchCursor(leafFrame);
+				ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 				RangePredicate rangePred = createRangePredicate(lowKey,
 						highKey, isForward, lowKeyInclusive, highKeyInclusive,
 						btree.getMultiComparator(), btree.getMultiComparator()
 								.getTypeTraits());
 				BTreeOpContext searchOpCtx = btree.createOpContext(
-						TreeIndexOp.TI_SEARCH, leafFrame, interiorFrame, null);
+						IndexOp.SEARCH, leafFrame, interiorFrame, null);
 				btree.search(rangeCursor, rangePred, searchOpCtx);
 
 				try {
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index 5eeb403..f55d086 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -29,24 +29,23 @@
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 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.api.IBTreeCursor;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
@@ -106,13 +105,13 @@
         MultiComparator btreeCmp = new MultiComparator(typeTraits, cmps);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);        
-        IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);       
-        IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
+        ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);       
+        ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
 
-        IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-        IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
-        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+        ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+        ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
         IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
         
@@ -205,7 +204,7 @@
        
         // ------- START VERIFICATION -----------
        
-        IBTreeCursor btreeCursor = new RangeSearchCursor(leafFrame);        
+        ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame)leafFrame);        
         FrameTupleReference searchKey = new FrameTupleReference();
         RangePredicate btreePred = new RangePredicate(true, searchKey, searchKey, true, true, btreeCmp, btreeCmp);
         
@@ -219,7 +218,7 @@
         IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
         tokenAccessor.reset(frame);
         
-        BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(TreeIndexOp.TI_SEARCH, leafFrame,
+        BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH, leafFrame,
                 interiorFrame, null);
         
         // verify created inverted lists one-by-one
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index 6d68ac2..81569fc 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -34,13 +34,12 @@
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
 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.api.IBTreeInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.NSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
@@ -91,13 +90,13 @@
     
     // btree frame factories
     private static TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);        
-    private static IBTreeLeafFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
-    private static IBTreeInteriorFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
+    private static ITreeIndexFrameFactory leafFrameFactory = new NSMLeafFrameFactory(tupleWriterFactory);
+    private static ITreeIndexFrameFactory interiorFrameFactory = new NSMInteriorFrameFactory(tupleWriterFactory);
     private static ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
     
     // btree frames
-    private static IBTreeLeafFrame leafFrame = leafFrameFactory.getFrame();
-    private static ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+    private static ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+    private static ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
     
     private static IFreePageManager freePageManager;
     
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index 66ae703..6ed84f2 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.NSMRTreeFrameFactory;
@@ -101,7 +101,7 @@
         IRTreeFrameFactory interiorFrameFactory = new NSMRTreeFrameFactory(interiorTupleWriterFactory, keyFieldCount);
         IRTreeFrameFactory leafFrameFactory = new NSMRTreeFrameFactory(leafTupleWriterFactory, keyFieldCount);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
         IRTreeFrame interiorFrame = interiorFrameFactory.getFrame();
         IRTreeFrame leafFrame = leafFrameFactory.getFrame();
@@ -126,7 +126,7 @@
         accessor.reset(hyracksFrame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        RTreeOpContext insertOpCtx = rtree.createOpContext(TreeIndexOp.TI_INSERT, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT, interiorFrame, leafFrame, metaFrame,
                 "unittest");
 
         Random rnd = new Random();
@@ -173,7 +173,7 @@
         // rtree.printTree(leafFrame, interiorFrame, recDescSers);
         // System.out.println();
 
-        RTreeOpContext searchOpCtx = rtree.createOpContext(TreeIndexOp.TI_SEARCH, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext searchOpCtx = rtree.createOpContext(IndexOp.SEARCH, interiorFrame, leafFrame, metaFrame,
                 "unittest");
         ArrayList<Rectangle> results = new ArrayList<Rectangle>();
         rtree.search(s, tuple, searchOpCtx, results);
@@ -243,7 +243,7 @@
         IRTreeFrameFactory interiorFrameFactory = new NSMRTreeFrameFactory(interiorTupleWriterFactory, keyFieldCount);
         IRTreeFrameFactory leafFrameFactory = new NSMRTreeFrameFactory(leafTupleWriterFactory, keyFieldCount);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
         IRTreeFrame interiorFrame = interiorFrameFactory.getFrame();
         IRTreeFrame leafFrame = leafFrameFactory.getFrame();
@@ -268,7 +268,7 @@
         accessor.reset(hyracksFrame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        RTreeOpContext insertOpCtx = rtree.createOpContext(TreeIndexOp.TI_INSERT, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT, interiorFrame, leafFrame, metaFrame,
                 "unittest");
 
         File datasetFile = new File("/home/salsubaiee/dataset.txt");
@@ -340,7 +340,7 @@
         // rtree.printTree(leafFrame, interiorFrame, recDescSers);
         // System.out.println();
 
-        RTreeOpContext searchOpCtx = rtree.createOpContext(TreeIndexOp.TI_SEARCH, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext searchOpCtx = rtree.createOpContext(IndexOp.SEARCH, interiorFrame, leafFrame, metaFrame,
                 "unittest");
 
         File querysetFile = new File("/home/salsubaiee/queryset.txt");
@@ -412,7 +412,7 @@
         // String stats = rtree.printStats();
         // print(stats);
 
-        RTreeOpContext deleteOpCtx = rtree.createOpContext(TreeIndexOp.TI_DELETE, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext deleteOpCtx = rtree.createOpContext(IndexOp.DELETE, interiorFrame, leafFrame, metaFrame,
                 "unittest");
 
         BufferedReader reader3 = new BufferedReader(new FileReader(datasetFile));
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index 59dfe10..e3d8aa4 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -34,7 +34,7 @@
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.TreeIndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeCursor;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeFrameFactory;
@@ -102,7 +102,7 @@
         IRTreeFrameFactory interiorFrameFactory = new NSMRTreeFrameFactory(interiorTupleWriterFactory, keyFieldCount);
         IRTreeFrameFactory leafFrameFactory = new NSMRTreeFrameFactory(leafTupleWriterFactory, keyFieldCount);
         ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.getFrame();
+        ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
 
         IRTreeFrame interiorFrame = interiorFrameFactory.getFrame();
         IRTreeFrame leafFrame = leafFrameFactory.getFrame();
@@ -127,7 +127,7 @@
         accessor.reset(hyracksFrame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        RTreeOpContext insertOpCtx = rtree.createOpContext(TreeIndexOp.TI_INSERT, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT, interiorFrame, leafFrame, metaFrame,
                 "unittest");
 
         Random rnd = new Random();
@@ -171,7 +171,7 @@
         }
 
         IRTreeCursor searchCursor = new SearchCursor(interiorFrame, leafFrame);
-        RTreeOpContext searchOpCtx = rtree.createOpContext(TreeIndexOp.TI_SEARCH, interiorFrame, leafFrame, metaFrame,
+        RTreeOpContext searchOpCtx = rtree.createOpContext(IndexOp.SEARCH, interiorFrame, leafFrame, metaFrame,
                 "cursortest");
         rtree.search(searchCursor, tuple, searchOpCtx);