Merged hyracks_dev_next into this branch.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@814 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 5ac4af2..eab76c2 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -30,7 +30,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -109,12 +108,12 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary index
-		primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
 		loadPrimaryIndexTest();
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index d2ee5e4..76679c2 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -110,12 +109,12 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary index
-		primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
 		loadPrimaryIndexTest();
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index e685894..f4c25fe 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -28,7 +28,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -104,12 +103,12 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary index
-		primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
 		loadPrimaryIndexTest();
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index c2a7201..386cba3 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -30,7 +30,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -139,17 +138,17 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary index
-		primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
 		// field, type and key declarations for secondary indexes
-		secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		secondaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 		secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 818dcd3..d334e32 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -136,17 +135,17 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary index
-		primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
 		// field, type and key declarations for secondary indexes
-		secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		secondaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 		secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
similarity index 87%
rename from hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
rename to hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
index 2206a26..fd558be 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
@@ -5,7 +5,6 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -21,7 +20,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
@@ -29,7 +28,7 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
 import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
 
-public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
+public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
 
     @Test
     public void tokenizerTest() throws Exception {
@@ -53,14 +52,12 @@
         IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
                 tokenFactory);
         int[] tokenFields = { 1 };
-        int[] projFields = { 0 };
+        int[] keyFields = { 0 };
         BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
-                tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
+                tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
new file mode 100644
index 0000000..19eec77
--- /dev/null
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -0,0 +1,333 @@
+/*
+ * 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.tests.invertedindex;
+
+import java.io.DataOutput;
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
+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.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+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.dataflow.std.sort.ExternalSortOperatorDescriptor;
+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.BTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
+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.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestInvertedIndexRegistryProvider;
+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 WordInvertedIndexTest extends AbstractIntegrationTest {
+    static {
+        TestStorageManagerComponentHolder.init(8192, 20, 20);
+    }
+
+    private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+    private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
+    private IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider = new TestInvertedIndexRegistryProvider();
+    private ITreeIndexOpHelperFactory btreeOpHelperFactory = new BTreeOpHelperFactory();
+
+    private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+    private final static String sep = System.getProperty("file.separator");
+    private final String dateString = simpleDateFormat.format(new Date());
+    private final String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree" + dateString;
+    private final String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree" + dateString;
+    private final String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists" + dateString;
+
+    private IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
+    private IFileSplitProvider btreeFileSplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(
+            NC1_ID, new FileReference(new File(btreeFileName))) });
+    private IFileSplitProvider invListsFileSplitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(invListsFileName))) });
+
+    // Primary BTree index.
+    private int primaryFieldCount = 2;
+    private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+    private int primaryKeyFieldCount = 1;
+    private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+    private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+    private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+            primaryTupleWriterFactory);
+    private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
+    private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+    // Inverted index BTree dictionary.
+    private int btreeFieldCount = 5;
+    private ITypeTrait[] btreeTypeTraits = new ITypeTrait[btreeFieldCount];
+    private int btreeKeyFieldCount = 1;
+    private IBinaryComparatorFactory[] btreeComparatorFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+    private TypeAwareTupleWriterFactory btreeTupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
+    private ITreeIndexFrameFactory btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(btreeTupleWriterFactory);
+    private ITreeIndexFrameFactory btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(btreeTupleWriterFactory);
+    private RecordDescriptor btreeRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE });
+
+    // Inverted index stuff.
+    private int invListElementFieldCount = 1;
+    private ITypeTrait[] invListsTypeTraits = new ITypeTrait[invListElementFieldCount];
+    private IBinaryComparatorFactory[] invListsComparatorFactories = new IBinaryComparatorFactory[invListElementFieldCount];
+    private RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+    private RecordDescriptor invListsRecDesc = new RecordDescriptor(
+            new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });    
+
+    // Tokenizer stuff.
+    private ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+    private IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+            tokenFactory);
+
+    @Before
+    public void setup() throws Exception {
+        // Field declarations and comparators for primary BTree index.
+        primaryTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+        primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+        // Field declarations and comparators for dictionary BTree.
+        btreeTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+        btreeTypeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        btreeTypeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        btreeTypeTraits[3] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        btreeTypeTraits[4] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        btreeComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+
+        // Field declarations and comparators for inverted lists.
+        invListsTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+        invListsComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+        loadPrimaryIndex();
+        printPrimaryIndex();
+        loadInvertedIndex();        
+    }
+
+    @Test
+    public void testConjunctiveSearcher() throws Exception {
+        IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
+        searchInvertedIndex("of", conjunctiveSearchModifierFactory);
+        searchInvertedIndex("3d", conjunctiveSearchModifierFactory);
+        searchInvertedIndex("of the human", conjunctiveSearchModifierFactory);
+    }
+
+    private IOperatorDescriptor createFileScanOp(JobSpecification spec) {
+        FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+                "data/cleanednumbereddblptitles.txt"))) };
+        IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
+        RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+        FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
+        return dblpTitleScanner;
+    }
+
+    private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
+        int[] fieldPermutation = { 0, 1 };
+        TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+                storageManager, treeIndexRegistryProvider, primaryFileSplitProvider, primaryInteriorFrameFactory,
+                primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+                btreeOpHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
+        return primaryBtreeBulkLoad;
+    }
+
+    private IOperatorDescriptor createScanKeyProviderOp(JobSpecification spec) throws HyracksDataException {
+        // build dummy tuple containing nothing
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+        DataOutput dos = tb.getDataOutput();
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+        tb.addFieldEndOffset();
+        ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+        return keyProviderOp;
+    }
+
+    private IOperatorDescriptor createPrimaryScanOp(JobSpecification spec) throws HyracksDataException {
+        int[] lowKeyFields = null; // - infinity
+        int[] highKeyFields = null; // + infinity
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+                storageManager, treeIndexRegistryProvider, primaryFileSplitProvider, primaryInteriorFrameFactory,
+                primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+                highKeyFields, true, true, btreeOpHelperFactory);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+        return primaryBtreeSearchOp;
+    }
+
+    private void loadPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        // Assuming that the data is pre-sorted on the key. No need to sort
+        // before bulk load.
+        IOperatorDescriptor fileScanOp = createFileScanOp(spec);
+        IOperatorDescriptor primaryBulkLoad = createPrimaryBulkLoadOp(spec);
+        spec.connect(new OneToOneConnectorDescriptor(spec), fileScanOp, 0, primaryBulkLoad, 0);
+        spec.addRoot(primaryBulkLoad);
+        runTest(spec);
+    }
+
+    private void printPrimaryIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
+        IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+
+    private IOperatorDescriptor createExternalSortOp(JobSpecification spec, int[] sortFields,
+            RecordDescriptor outputRecDesc) {
+        ExternalSortOperatorDescriptor externalSortOp = new ExternalSortOperatorDescriptor(spec, 1000, sortFields,
+                new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
+                        IntegerBinaryComparatorFactory.INSTANCE }, outputRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, externalSortOp, NC1_ID);
+        return externalSortOp;
+    }
+
+    private IOperatorDescriptor createBinaryTokenizerOp(JobSpecification spec, int[] tokenFields, int[] keyFields) {
+        BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
+                tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
+        return binaryTokenizer;
+    }
+
+    private IOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec, int[] fieldPermutation) {
+        InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new InvertedIndexBulkLoadOperatorDescriptor(spec,
+                storageManager, fieldPermutation, btreeFileSplitProvider, treeIndexRegistryProvider,
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories,
+                BTree.DEFAULT_FILL_FACTOR, btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider,
+                invListsTypeTraits, invListsComparatorFactories);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexBulkLoadOp, NC1_ID);
+        return invIndexBulkLoadOp;
+    }
+
+    public void loadInvertedIndex() throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
+        IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
+        int[] tokenFields = { 1 };
+        int[] keyFields = { 0 };
+        IOperatorDescriptor binaryTokenizerOp = createBinaryTokenizerOp(spec, tokenFields, keyFields);
+        int[] sortFields = { 0, 1 };
+        IOperatorDescriptor externalSortOp = createExternalSortOp(spec, sortFields, tokenizerRecDesc);
+        int[] fieldPermutation = { 0, 1 };
+        IOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec, fieldPermutation);
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, binaryTokenizerOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizerOp, 0, externalSortOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), externalSortOp, 0, invIndexBulkLoadOp, 0);
+        spec.addRoot(invIndexBulkLoadOp);
+        runTest(spec);
+    }
+
+    private IOperatorDescriptor createQueryProviderOp(JobSpecification spec, String queryString)
+            throws HyracksDataException {
+        // Build tuple with exactly one field, which is the query,
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+        DataOutput dos = tb.getDataOutput();
+        tb.reset();
+        UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString, dos);
+        tb.addFieldEndOffset();
+        ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
+        RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
+        ConstantTupleSourceOperatorDescriptor queryProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+                queryRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, queryProviderOp, NC1_ID);
+        return queryProviderOp;
+    }
+
+    private IOperatorDescriptor createInvertedIndexSearchOp(JobSpecification spec,
+            IInvertedIndexSearchModifierFactory searchModifierFactory) {
+        InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(spec, 0,
+                storageManager, btreeFileSplitProvider, treeIndexRegistryProvider, btreeInteriorFrameFactory,
+                btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeOpHelperFactory,
+                invListsFileSplitProvider, invIndexRegistryProvider, invListsTypeTraits, invListsComparatorFactories,
+                searchModifierFactory, tokenizerFactory, invListsRecDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexSearchOp, NC1_ID);
+        return invIndexSearchOp;
+    }
+
+    public void searchInvertedIndex(String queryString, IInvertedIndexSearchModifierFactory searchModifierFactory)
+            throws Exception {
+        JobSpecification spec = new JobSpecification();
+        IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
+        IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
+        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+        spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
+        spec.addRoot(printer);
+        runTest(spec);
+    }
+}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 4b31862..d4496cb 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -112,11 +111,11 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary R-tree index
-		primaryTypeTraits[0] = new TypeTrait(8);
-		primaryTypeTraits[1] = new TypeTrait(8);
-		primaryTypeTraits[2] = new TypeTrait(8);
-		primaryTypeTraits[3] = new TypeTrait(8);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
 		primaryComparatorFactories[1] = primaryComparatorFactories[0];
 		primaryComparatorFactories[2] = primaryComparatorFactories[0];
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index 1b1d760..183c869 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -28,7 +28,6 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
@@ -106,11 +105,11 @@
 	@Before
 	public void setup() throws Exception {
 		// field, type and key declarations for primary R-tree index
-		primaryTypeTraits[0] = new TypeTrait(8);
-		primaryTypeTraits[1] = new TypeTrait(8);
-		primaryTypeTraits[2] = new TypeTrait(8);
-		primaryTypeTraits[3] = new TypeTrait(8);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
 		primaryComparatorFactories[1] = primaryComparatorFactories[0];
 		primaryComparatorFactories[2] = primaryComparatorFactories[0];
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 5a24036..c484c2d 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -25,12 +25,10 @@
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -47,8 +45,8 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+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.dataflow.BTreeOpHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -89,35 +87,6 @@
 			"ddMMyy-hhmmssSS");
 	private final static String sep = System.getProperty("file.separator");
 
-	// field, type and key declarations for primary R-tree index
-	private int primaryFieldCount = 5;
-	private int primaryKeyFieldCount = 4;
-	private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
-	private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
-
-	private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
-			primaryTypeTraits);
-
-	private ITreeIndexFrameFactory primaryInteriorFrameFactory;
-	private ITreeIndexFrameFactory primaryLeafFrameFactory;
-
-	private static String primaryRTreeName = "primary"
-			+ simpleDateFormat.format(new Date());
-	private static String primaryFileName = System
-			.getProperty("java.io.tmpdir") + sep + primaryRTreeName;
-
-	private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
-			new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
-					primaryFileName))) });
-
-	private RecordDescriptor primaryRecDesc = new RecordDescriptor(
-			new ISerializerDeserializer[] {
-					DoubleSerializerDeserializer.INSTANCE,
-					DoubleSerializerDeserializer.INSTANCE,
-					DoubleSerializerDeserializer.INSTANCE,
-					DoubleSerializerDeserializer.INSTANCE,
-					UTF8StringSerializerDeserializer.INSTANCE });
-
 	// field, type and key declarations for primary B-tree index
 	private int primaryBTreeFieldCount = 10;
 	private ITypeTrait[] primaryBTreeTypeTraits = new ITypeTrait[primaryBTreeFieldCount];
@@ -183,45 +152,26 @@
 					UTF8StringSerializerDeserializer.INSTANCE });
 
 	@Before
-	public void setup() throws Exception {
-		// field, type and key declarations for primary R-tree index
-		primaryTypeTraits[0] = new TypeTrait(8);
-		primaryTypeTraits[1] = new TypeTrait(8);
-		primaryTypeTraits[2] = new TypeTrait(8);
-		primaryTypeTraits[3] = new TypeTrait(8);
-		primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
-		primaryComparatorFactories[1] = primaryComparatorFactories[0];
-		primaryComparatorFactories[2] = primaryComparatorFactories[0];
-		primaryComparatorFactories[3] = primaryComparatorFactories[0];
-
-		IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
-				.comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
-		
-		primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
-				primaryTupleWriterFactory, primaryValueProviderFactories);
-		primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(
-				primaryTupleWriterFactory, primaryValueProviderFactories);
-		
+	public void setup() throws Exception {	
 		// field, type and key declarations for primary B-tree index
-		primaryBTreeTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryBTreeTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryBTreeTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryBTreeTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryBTreeTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryBTreeTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-		primaryBTreeTypeTraits[6] = new TypeTrait(8);
-		primaryBTreeTypeTraits[7] = new TypeTrait(8);
-		primaryBTreeTypeTraits[8] = new TypeTrait(8);
-		primaryBTreeTypeTraits[9] = new TypeTrait(8);
+		primaryBTreeTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryBTreeTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryBTreeTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryBTreeTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryBTreeTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryBTreeTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
+		primaryBTreeTypeTraits[6] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryBTreeTypeTraits[7] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryBTreeTypeTraits[8] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		primaryBTreeTypeTraits[9] = ITypeTrait.DOUBLE_TYPE_TRAIT;
 		primaryBTreeComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
 		// field, type and key declarations for secondary indexes
-		secondaryTypeTraits[0] = new TypeTrait(8);
-		secondaryTypeTraits[1] = new TypeTrait(8);
-		secondaryTypeTraits[2] = new TypeTrait(8);
-		secondaryTypeTraits[3] = new TypeTrait(8);
-		secondaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+		secondaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		secondaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		secondaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		secondaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+		secondaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
 		secondaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
 		secondaryComparatorFactories[1] = secondaryComparatorFactories[0];
 		secondaryComparatorFactories[2] = secondaryComparatorFactories[0];
@@ -239,7 +189,6 @@
 		secondaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(
 				secondaryTupleWriterFactory, secondaryValueProviderFactories);
 		
-		loadPrimaryIndexTest();
 		loadPrimaryBTreeIndexTest();
 		loadSecondaryIndexTest();
 	}
@@ -316,54 +265,12 @@
 		runTest(spec);
 	}
 
-	public void loadPrimaryIndexTest() throws Exception {
-		JobSpecification spec = new JobSpecification();
-
-		FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID,
-				new FileReference(new File("data/spatial.txt"))) };
-		IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(
-				objectsSplits);
-		RecordDescriptor objectsDesc = new RecordDescriptor(
-				new ISerializerDeserializer[] {
-						DoubleSerializerDeserializer.INSTANCE,
-						DoubleSerializerDeserializer.INSTANCE,
-						DoubleSerializerDeserializer.INSTANCE,
-						DoubleSerializerDeserializer.INSTANCE,
-						UTF8StringSerializerDeserializer.INSTANCE });
-
-		FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(
-				spec, objectsSplitProvider,
-				new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
-						DoubleParserFactory.INSTANCE,
-						DoubleParserFactory.INSTANCE,
-						DoubleParserFactory.INSTANCE,
-						DoubleParserFactory.INSTANCE,
-						UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				objScanner, NC1_ID);
-
-		int[] fieldPermutation = { 0, 1, 2, 3, 4 };
-		TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
-				spec, storageManager, treeIndexRegistryProvider,
-				primaryRTreeSplitProvider, primaryInteriorFrameFactory,
-				primaryLeafFrameFactory, primaryTypeTraits,
-				primaryComparatorFactories,
-				fieldPermutation, 0.7f, opHelperFactory);
-		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
-				primaryRTreeBulkLoad, NC1_ID);
-
-		spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0,
-				primaryRTreeBulkLoad, 0);
-
-		spec.addRoot(primaryRTreeBulkLoad);
-		runTest(spec);
-	}
 
 	public void loadSecondaryIndexTest() throws Exception {
 		JobSpecification spec = new JobSpecification();
 
 		// build dummy tuple containing nothing
-		ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+		ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryBTreeKeyFieldCount * 2);
 		DataOutput dos = tb.getDataOutput();
 
 		tb.reset();
@@ -390,7 +297,7 @@
 				treeIndexRegistryProvider, primaryBTreeSplitProvider,
 				primaryBTreeInteriorFrameFactory, primaryBTreeLeafFrameFactory,
 				primaryBTreeTypeTraits, primaryBTreeComparatorFactories, true,
-				lowKeyFields, highKeyFields, true, true, opHelperFactory);
+				lowKeyFields, highKeyFields, true, true, bTreeopHelperFactory);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				primaryBTreeSearchOp, NC1_ID);
 
@@ -456,9 +363,7 @@
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
 				secondaryRTreeSearchOp, NC1_ID);
 
-        IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
-                createTempFile().getAbsolutePath()) });
-        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+		PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
 		PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
 				NC1_ID);
 
@@ -473,9 +378,6 @@
 
 	@AfterClass
 	public static void cleanup() throws Exception {
-		File primary = new File(primaryFileName);
-		primary.deleteOnExit();
-
 		File primaryBTree = new File(primaryBTreeFileName);
 		primaryBTree.deleteOnExit();
 
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index 3f7bc30..8e88c18 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -178,14 +178,14 @@
 
                     // number of fields we decided to use for compression of
                     // this keyPartition
-                    int numFieldsToCompress = keyPartitions.get(kpIndex).maxPmiIndex + 1;
+                    int fieldCountToCompress = keyPartitions.get(kpIndex).maxPmiIndex + 1;
                     int segmentStart = keyPartitions.get(kpIndex).firstTupleIndex;
                     int tuplesInSegment = 1;
 
                     // System.out.println("PROCESSING KEYPARTITION: " + kpIndex
                     // + " RANGE: " + keyPartitions.get(kpIndex).firstRecSlotNum
                     // + " " + keyPartitions.get(kpIndex).lastRecSlotNum +
-                    // " FIELDSTOCOMPRESS: " + numFieldsToCompress);
+                    // " FIELDSTOCOMPRESS: " + fieldCountToCompress);
 
                     FieldPrefixTupleReference prevTuple = new FieldPrefixTupleReference(tupleWriter
                             .createTupleReference());
@@ -198,10 +198,10 @@
                         prevTuple.resetByTupleIndex(frame, i - 1);
                         tuple.resetByTupleIndex(frame, i);
 
-                        // check if tuples match in numFieldsToCompress of their
+                        // check if tuples match in fieldCountToCompress of their
                         // first fields
                         int prefixFieldsMatch = 0;
-                        for (int j = 0; j < numFieldsToCompress; j++) {
+                        for (int j = 0; j < fieldCountToCompress; j++) {
                             if (cmps[j].compare(pageArray, prevTuple.getFieldStart(j), prevTuple.getFieldLength(j),
                                     pageArray, tuple.getFieldStart(j), tuple.getFieldLength(j)) == 0)
                                 prefixFieldsMatch++;
@@ -212,7 +212,7 @@
                         // the two tuples must match in exactly the number of
                         // fields we decided to compress for this keyPartition
                         int processSegments = 0;
-                        if (prefixFieldsMatch == numFieldsToCompress)
+                        if (prefixFieldsMatch == fieldCountToCompress)
                             tuplesInSegment++;
                         else
                             processSegments++;
@@ -223,7 +223,7 @@
                         for (int r = 0; r < processSegments; r++) {
                             // compress current segment and then start new
                             // segment
-                            if (tuplesInSegment < occurrenceThreshold || numFieldsToCompress <= 0) {
+                            if (tuplesInSegment < occurrenceThreshold || fieldCountToCompress <= 0) {
                                 // segment does not have at least
                                 // occurrenceThreshold tuples, so write tuples
                                 // uncompressed
@@ -240,19 +240,19 @@
                                 // extract prefix, write prefix tuple to buffer,
                                 // and set prefix slot
                                 newPrefixSlots[newPrefixSlots.length - 1 - prefixTupleIndex] = slotManager
-                                        .encodeSlotFields(numFieldsToCompress, prefixFreeSpace);
+                                        .encodeSlotFields(fieldCountToCompress, prefixFreeSpace);
                                 // int tmp = freeSpace;
                                 // prevRec.reset();
                                 // System.out.println("SOURCE CONTENTS: " +
                                 // buf.getInt(prevRec.getFieldOff()) + " " +
                                 // buf.getInt(prevRec.getFieldOff()+4));
-                                prefixFreeSpace += tupleWriter.writeTupleFields(prevTuple, 0, numFieldsToCompress,
+                                prefixFreeSpace += tupleWriter.writeTupleFields(prevTuple, 0, fieldCountToCompress,
                                         byteBuffer, prefixFreeSpace);
                                 // System.out.println("WRITING PREFIX RECORD " +
                                 // prefixSlotNum + " AT " + tmp + " " +
                                 // freeSpace);
                                 // System.out.print("CONTENTS: ");
-                                // for(int x = 0; x < numFieldsToCompress; x++)
+                                // for(int x = 0; x < fieldCountToCompress; x++)
                                 // System.out.print(buf.getInt(tmp + x*4) +
                                 // " ");
                                 // System.out.println();
@@ -264,8 +264,8 @@
                                     tupleToWrite.resetByTupleIndex(frame, currTupleIndex);
                                     newTupleSlots[tupleCount - 1 - currTupleIndex] = slotManager.encodeSlotFields(
                                             prefixTupleIndex, tupleFreeSpace);
-                                    tupleFreeSpace += tupleWriter.writeTupleFields(tupleToWrite, numFieldsToCompress,
-                                            fieldCount - numFieldsToCompress, byteBuffer, tupleFreeSpace);
+                                    tupleFreeSpace += tupleWriter.writeTupleFields(tupleToWrite, fieldCountToCompress,
+                                            fieldCount - fieldCountToCompress, byteBuffer, tupleFreeSpace);
                                 }
 
                                 prefixTupleIndex++;
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 8cfd5d6..48ee8d4 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
@@ -18,7 +18,6 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -30,16 +29,16 @@
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 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.BTreeRangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 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;
 
 public class BTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
@@ -62,7 +61,7 @@
     private MultiComparator highKeySearchCmp;
     private ITreeIndexCursor cursor;
     private ITreeIndexFrame cursorFrame;
-    private BTreeOpContext opCtx;
+    private ITreeIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
 
@@ -99,33 +98,9 @@
             treeIndexOpHelper.init();
             btree = (BTree) treeIndexOpHelper.getTreeIndex();
 
-            // construct range predicate
-
-            // TODO: Can we construct the multicmps using helper methods?
-            int lowKeySearchFields = btree.getMultiComparator().getComparators().length;
-            int highKeySearchFields = btree.getMultiComparator().getComparators().length;
-            if (lowKey != null)
-                lowKeySearchFields = lowKey.getFieldCount();
-            if (highKey != null)
-                highKeySearchFields = highKey.getFieldCount();
-
-            IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
-            for (int i = 0; i < lowKeySearchFields; i++) {
-                lowKeySearchComparators[i] = btree.getMultiComparator().getComparators()[i];
-            }
-            lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
-
-            if (lowKeySearchFields == highKeySearchFields) {
-                highKeySearchCmp = lowKeySearchCmp;
-            } else {
-                IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
-                for (int i = 0; i < highKeySearchFields; i++) {
-                    highKeySearchComparators[i] = btree.getMultiComparator().getComparators()[i];
-                }
-                highKeySearchCmp = new MultiComparator(highKeySearchComparators);
-
-            }
-
+            // Construct range predicate.
+            lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getMultiComparator(), lowKey);
+            highKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getMultiComparator(), highKey);
             rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
 
@@ -134,9 +109,7 @@
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
             appender.reset(writeBuffer, true);
-
-            opCtx = btree.createOpContext(IndexOp.SEARCH);
-
+            indexAccessor = btree.createAccessor();
         } catch (Exception e) {
             treeIndexOpHelper.deinit();
             throw new HyracksDataException(e);
@@ -179,7 +152,7 @@
                 rangePred.setHighKey(highKey, highKeyInclusive);
 
                 cursor.reset();
-                btree.search(cursor, rangePred, opCtx);
+                indexAccessor.search(cursor, rangePred);
                 writeSearchResults();
             }
         } catch (Exception e) {
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java
index 1e09658..c568f6f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java
@@ -20,7 +20,6 @@
 public class BTreeException extends TreeIndexException {
 
     protected static final long serialVersionUID = 1L;
-    protected boolean handled = false;
 
     public BTreeException(Exception e) {
         super(e);
@@ -29,12 +28,4 @@
     public BTreeException(String message) {
         super(message);
     }
-
-    public void setHandled(boolean handled) {
-        this.handled = handled;
-    }
-
-    public boolean getHandled() {
-        return handled;
-    }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 0334a7f..56e0774 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -268,11 +268,11 @@
         int newTupleBytes = 0;
         
         int numPrefixFields = frameTuple.getNumPrefixFields();
-        int numFields = frameTuple.getFieldCount();
+        int fieldCount = frameTuple.getFieldCount();
         if (numPrefixFields != 0) {
             // Check the space requirements for updating the suffix of the original tuple.            
             oldTupleBytes = frameTuple.getSuffixTupleSize();
-            newTupleBytes = tupleWriter.bytesRequired(newTuple, numPrefixFields, numFields - numPrefixFields); 
+            newTupleBytes = tupleWriter.bytesRequired(newTuple, numPrefixFields, fieldCount - numPrefixFields); 
         } else {
             // The original tuple is uncompressed.
             oldTupleBytes = frameTuple.getTupleSize();
@@ -308,18 +308,18 @@
         int suffixTupleStartOff = slotManager.decodeSecondSlotField(tupleSlot);                
         
         frameTuple.resetByTupleIndex(this, tupleIndex);
-        int numFields = frameTuple.getFieldCount();
+        int fieldCount = frameTuple.getFieldCount();
         int numPrefixFields = frameTuple.getNumPrefixFields();
         int oldTupleBytes = frameTuple.getSuffixTupleSize();
         int bytesWritten = 0;        
         
         if (inPlace) {
             // Overwrite the old tuple suffix in place.
-            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, numFields - numPrefixFields, buf, suffixTupleStartOff);
+            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields, buf, suffixTupleStartOff);
         } else {
             // Insert the new tuple suffix at the end of the free space, and change the slot value (effectively "deleting" the old tuple).
             int newSuffixTupleStartOff = buf.getInt(freeSpaceOff);
-            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, numFields - numPrefixFields, buf, newSuffixTupleStartOff);
+            bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields, buf, newSuffixTupleStartOff);
             // Update slot value using the same prefix slot num.
             slotManager.setSlot(tupleSlotOff, slotManager.encodeSlotFields(prefixSlotNum, newSuffixTupleStartOff));
             // Update contiguous free space pointer.
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 3952f5d..50e3c0b 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
@@ -30,14 +30,16 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrame;
 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.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.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.PageAllocationException;
 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.impls.TreeDiskOrderScanCursor;
@@ -118,10 +120,8 @@
         ctx.freePages.clear();
     }
     
-    @Override
-    public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException {
+    private void diskOrderScan(ITreeIndexCursor icursor, BTreeOpContext ctx) throws HyracksDataException {
         TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
-        BTreeOpContext ctx = (BTreeOpContext) ictx;
         ctx.reset();
 
         int currentPageId = rootPage;
@@ -143,9 +143,10 @@
         }
     }
 
-    public void search(ITreeIndexCursor cursor, RangePredicate pred, BTreeOpContext ctx) throws Exception {
+    private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
+            throws TreeIndexException, HyracksDataException, PageAllocationException {
         ctx.reset();
-        ctx.pred = pred;
+        ctx.pred = (RangePredicate) searchPred;
         ctx.cursor = cursor;
         // simple index scan
         if (ctx.pred.getLowKeyComparator() == null) {
@@ -204,7 +205,7 @@
         }
     }
     
-    private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+    private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
         // Make sure the root is always in the same page.
         ICachedPage leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, ctx.splitKey.getLeftPage()),
                 false);
@@ -250,8 +251,7 @@
         }
     }
     
-    private void insertUpdateOrDelete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
-        BTreeOpContext ctx = (BTreeOpContext) ictx;
+    private void insertUpdateOrDelete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
         ctx.reset();
         ctx.pred.setLowKeyComparator(cmp);
         ctx.pred.setHighKeyComparator(cmp);
@@ -287,25 +287,22 @@
         }
     }
     
-    @Override
-    public void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
-        insertUpdateOrDelete(tuple, ictx);
+    private void insert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
+        insertUpdateOrDelete(tuple, ctx);
     }
 
-    @Override
-    public void update(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+    private void update(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
         // This call only allows updating of non-key fields.
         // Updating a tuple's key necessitates deleting the old entry, and inserting the new entry.
         // The user of the BTree is responsible for dealing with non-key updates (i.e., doing a delete + insert). 
         if (fieldCount == cmp.getKeyFieldCount()) {
             throw new BTreeNotUpdateableException("Cannot perform updates when the entire tuple forms the key.");
         }
-        insertUpdateOrDelete(tuple, ictx);
+        insertUpdateOrDelete(tuple, ctx);
     }
     
-    @Override
-    public void delete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
-        insertUpdateOrDelete(tuple, ictx);
+    private void delete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
+        insertUpdateOrDelete(tuple, ctx);
     }
     
     private void insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
@@ -363,7 +360,7 @@
                     true);
             rightNode.acquireWriteLatch();
             try {
-                IBTreeLeafFrame rightFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();                
+                IBTreeLeafFrame rightFrame = ctx.createLeafFrame();                
                 rightFrame.setPage(rightNode);
                 rightFrame.initBuffer((byte) 0);
                 rightFrame.setMultiComparator(cmp);
@@ -466,7 +463,7 @@
                 ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true);
                 rightNode.acquireWriteLatch();
                 try {
-                    IBTreeFrame rightFrame = (IBTreeFrame)interiorFrameFactory.createFrame();
+                    IBTreeFrame rightFrame = ctx.createInteriorFrame();
                     rightFrame.setPage(rightNode);
                     rightFrame.initBuffer((byte) ctx.interiorFrame.getLevel());
                     rightFrame.setMultiComparator(cmp);
@@ -523,7 +520,7 @@
         }
         
         // Leaf will become empty. 
-        IBTreeLeafFrame siblingFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+        IBTreeLeafFrame siblingFrame = ctx.createLeafFrame();
         siblingFrame.setMultiComparator(cmp);
         ICachedPage leftNode = null;
         ICachedPage rightNode = null;
@@ -661,15 +658,10 @@
         return isConsistent;
     }
 
-    private void performOp(int pageId, ICachedPage parent, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+    private void performOp(int pageId, ICachedPage parent, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
         ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
         ctx.interiorFrame.setPage(node);
         
-        //System.out.println("PAGEID: " + pageId);
-        //System.out.println("NODE:   " + node);
-        //System.out.println("LEVEL:  " + ctx.interiorFrame.getLevel() + " " + ctx.interiorFrame.isLeaf());
-        //System.out.println("-------------------------");
-        
         // this check performs an unprotected read in the page
         // the following could happen: TODO fill out
         boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
@@ -792,25 +784,24 @@
                 }
             }
         } catch (TreeIndexException e) {
-            //e.printStackTrace();
-            if (!e.getHandled()) {
+            if (!ctx.exceptionHandled) {
                 releaseLatch(node, ctx.op, unsafeIsLeaf);
                 bufferCache.unpin(node);
-                e.setHandled(true);
+                ctx.exceptionHandled = true;
+            }
+            throw e;
+        } catch (PageAllocationException e) {
+            if (!ctx.exceptionHandled) {
+                releaseLatch(node, ctx.op, unsafeIsLeaf);
+                bufferCache.unpin(node);
+                ctx.exceptionHandled = true;
             }
             throw e;
         } catch (Exception e) {
-            //e.printStackTrace();
-            // This could be caused, e.g. by a failure to pin a new node during a split.
             releaseLatch(node, ctx.op, unsafeIsLeaf);
             bufferCache.unpin(node);
-            BTreeException propException = new BTreeException(e);
-            propException.setHandled(true);
-            // propagate a BTreeException,
-            // indicating that the parent node
-            // must not be unlatched and
-            // unpinned
-            throw propException;
+            BTreeException wrappedException = new BTreeException(e);
+            throw wrappedException;
         }
     }
 
@@ -828,7 +819,7 @@
         private final ITreeIndexTupleWriter tupleWriter;
 
         public BulkLoadContext(float fillFactor, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
-                ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) throws HyracksDataException {
+                ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) throws HyracksDataException, PageAllocationException {
 
         	leafFrame.setMultiComparator(cmp);
         	interiorFrame.setMultiComparator(cmp);
@@ -859,7 +850,7 @@
             nodeFrontiers.add(leafFrontier);
         }
 
-        private void addLevel() throws HyracksDataException {
+        private void addLevel() throws HyracksDataException, PageAllocationException {
             NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
             frontier.pageId = freePageManager.getFreePage(metaFrame);
             frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);
@@ -871,7 +862,7 @@
         }
     }
 
-    private void propagateBulk(BulkLoadContext ctx, int level) throws HyracksDataException {
+    private void propagateBulk(BulkLoadContext ctx, int level) throws HyracksDataException, PageAllocationException {
 
         if (ctx.splitKey.getBuffer() == null)
             return;
@@ -917,7 +908,7 @@
 
     // assumes btree has been created and opened
     @Override
-    public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException, HyracksDataException {
+    public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException, HyracksDataException, PageAllocationException {
         IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
     	if (!isEmptyTree(leafFrame)) {
     		throw new BTreeException("Trying to Bulk-load a non-empty BTree.");
@@ -925,13 +916,12 @@
     	
         BulkLoadContext ctx = new BulkLoadContext(fillFactor, leafFrame,
                 (IBTreeInteriorFrame)interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory().createFrame(), cmp);
-        ctx.nodeFrontiers.get(0).lastTuple.setFieldCount(fieldCount);
         ctx.splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
         return ctx;
     }
 
     @Override
-    public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
+    public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException, PageAllocationException {
         BulkLoadContext ctx = (BulkLoadContext) ictx;
         NodeFrontier leafFrontier = ctx.nodeFrontiers.get(0);
         IBTreeLeafFrame leafFrame = ctx.leafFrame;
@@ -1006,11 +996,9 @@
         }
     }
 
-    @Override
-    public BTreeOpContext createOpContext(IndexOp op) {
-        return new BTreeOpContext(op, (IBTreeLeafFrame) leafFrameFactory.createFrame(),
-                (IBTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
-                        .createFrame(), cmp);
+    private BTreeOpContext createOpContext() {
+        return new BTreeOpContext(leafFrameFactory, interiorFrameFactory, freePageManager.getMetaDataFrameFactory()
+                .createFrame(), cmp);
     }
     
     public ITreeIndexFrameFactory getInteriorFrameFactory() {
@@ -1122,4 +1110,50 @@
             e.printStackTrace();
         }
     }
+
+    @Override
+    public ITreeIndexAccessor createAccessor() {
+        return new BTreeAccessor(this);
+    }
+    
+    private class BTreeAccessor implements ITreeIndexAccessor {
+        private BTree btree;
+        private BTreeOpContext ctx;
+        
+        public BTreeAccessor(BTree btree) {
+            this.btree = btree;
+            this.ctx = btree.createOpContext();
+        }
+        
+        @Override
+        public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+            ctx.reset(IndexOp.INSERT);
+            btree.insert(tuple, ctx);
+        }
+
+        @Override
+        public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+            ctx.reset(IndexOp.UPDATE);
+            btree.update(tuple, ctx);
+        }
+
+        @Override
+        public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+            ctx.reset(IndexOp.DELETE);
+            btree.delete(tuple, ctx);
+        }
+
+        @Override
+        public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+                TreeIndexException, PageAllocationException {
+            ctx.reset(IndexOp.SEARCH);
+            btree.search(cursor, searchPred, ctx);
+        }
+
+        @Override
+        public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
+            ctx.reset(IndexOp.DISKORDERSCAN);
+            btree.diskOrderScan(cursor, ctx);
+        }
+    }
 }
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 82c747c..07c645c 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
@@ -19,40 +19,45 @@
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
 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.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IntArrayList;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.LongArrayList;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 
-public final class BTreeOpContext implements IIndexOpContext {
-    private final int INIT_ARRAYLIST_SIZE = 6;    
-    public final IBTreeLeafFrame leafFrame;
-    public final IBTreeInteriorFrame interiorFrame;
-    public final ITreeIndexMetaDataFrame metaFrame;
+public class BTreeOpContext implements IIndexOpContext {
+    private final int INIT_ARRAYLIST_SIZE = 6; 
+    protected ITreeIndexFrameFactory leafFrameFactory;
+    protected ITreeIndexFrameFactory interiorFrameFactory;
+    public IBTreeLeafFrame leafFrame;
+    public IBTreeInteriorFrame interiorFrame;
+    public ITreeIndexMetaDataFrame metaFrame;
     public IndexOp op;
     public ITreeIndexCursor cursor;
     public BTreeCursorInitialState cursorInitialState;
     public RangePredicate pred;
-    public BTreeSplitKey splitKey;
-    public int opRestarts = 0;
+    public BTreeSplitKey splitKey;    
     public LongArrayList pageLsns;
     public IntArrayList smPages;
     public IntArrayList freePages;
-
-    public BTreeOpContext(IndexOp op, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
-            ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) {        
+    public int opRestarts = 0;
+    public boolean exceptionHandled;
+    
+    public BTreeOpContext(ITreeIndexFrameFactory leafFrameFactory, ITreeIndexFrameFactory interiorFrameFactory,
+            ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) {
+        this.leafFrameFactory = leafFrameFactory;
+        this.leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
         if (leafFrame != null) {
-        	leafFrame.setMultiComparator(cmp);
+            leafFrame.setMultiComparator(cmp);
         }
-        this.leafFrame = leafFrame;
+        this.interiorFrameFactory = interiorFrameFactory;
+        this.interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
         if (interiorFrame != null) {
-        	interiorFrame.setMultiComparator(cmp);
+            interiorFrame.setMultiComparator(cmp);
         }
-        this.interiorFrame = interiorFrame;
         this.metaFrame = metaFrame;
         this.pageLsns = new LongArrayList(INIT_ARRAYLIST_SIZE, INIT_ARRAYLIST_SIZE);
-        reset(op);
     }
 
     public void reset() {
@@ -63,6 +68,7 @@
         if (smPages != null)
             smPages.clear();
         opRestarts = 0;
+        exceptionHandled = false;
     }
 
     @Override
@@ -86,6 +92,15 @@
                 splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
             }
         }
-        this.op = newOp;
+        op = newOp;
+        exceptionHandled = false;
+    }
+
+    public IBTreeLeafFrame createLeafFrame() {
+        return (IBTreeLeafFrame) leafFrameFactory.createFrame();
+    }
+
+    public IBTreeInteriorFrame createInteriorFrame() {
+        return (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
     }
 }
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 1072bfb..3520ec9 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -174,7 +174,7 @@
     }
 
     @Override
-    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws Exception {
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         // in case open is called multiple times without closing
         if (page != null) {
             page.releaseReadLatch();
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
index 79945b6..625fa02 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.hyracks.storage.am.btree.util;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
@@ -59,12 +58,4 @@
             }
         }
     }
-    
-    public static MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
-    	IBinaryComparator[] cmps = new IBinaryComparator[cmpFactories.length];
-    	for (int i = 0; i < cmpFactories.length; i++) {
-    		cmps[i] = cmpFactories[i].createBinaryComparator(); 
-    	}
-    	return new MultiComparator(cmps);
-    }
 }
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java
index 045ff9d..a7901c8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java
@@ -4,7 +4,7 @@
 
 public interface IFreePageManager {
 	public int getFreePage(ITreeIndexMetaDataFrame metaFrame)
-			throws HyracksDataException;
+			throws HyracksDataException, PageAllocationException;
 
 	public void addFreePage(ITreeIndexMetaDataFrame metaFrame, int freePage)
 			throws HyracksDataException;
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
index 83d0639..91b220d 100644
--- 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
@@ -17,12 +17,12 @@
 
 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;
 
 /**
  * Interface describing the operations of tree-based index structures. Indexes
  * implementing this interface can easily reuse the tree index operators for
  * dataflow. We assume that indexes store tuples with a fixed number of fields.
+ * Users must perform operations on an ITreeIndex via an ITreeIndexAccessor.
  */
 public interface ITreeIndex {
 
@@ -51,69 +51,14 @@
 	public void close();
 
 	/**
-	 * Creates an operation context for a given index operation
-	 * (insert/delete/update/search/diskorderscan). An operation context
-	 * maintains a cache of objects used during the traversal of the tree index.
-	 * The context is intended to be reused for multiple subsequent operations
-	 * by the same user/thread. An index operation context is stateful, and
-	 * therefore, should not be shared among two threads.
+	 * Creates an index accessor for performing operations on this index.
+	 * (insert/delete/update/search/diskorderscan). An ITreeIndexAccessor is not
+	 * thread safe, but different ITreeIndexAccessors can concurrently operate
+	 * on the same ITreeIndex
 	 * 
-	 * @param indexOp
-	 *            Intended index operation.
-	 * 
-	 * @returns IITreeIndexOpContext Operation context for the desired index
-	 *          operation.
+	 * @returns ITreeIndexAccessor A tree index accessor for this tree.
 	 */
-	public IIndexOpContext createOpContext(IndexOp op);
-	
-	/**
-	 * Inserts the given tuple into the index using an existing operation
-	 * context.
-	 * 
-	 * @param tuple
-	 *            Tuple to be inserted.
-	 * @param ictx
-	 *            Existing operation context.
-	 * @throws HyracksDataException
-	 *             If the BufferCache throws while un/pinning or un/latching.
-	 * @throws TreeIndexException
-	 *             If an index-specific constraint is violated, e.g., the key
-	 *             already exists.
-	 */
-	public void insert(ITupleReference tuple, IIndexOpContext ictx)
-			throws HyracksDataException, TreeIndexException;
-
-	/**
-	 * Updates the tuple in the index matching the given tuple with the new
-	 * contents in the given tuple.
-	 * 
-	 * @param tuple
-	 *            Tuple whose match in the index is to be update with the given
-	 *            tuples contents.
-	 * @param ictx
-	 *            Existing operation context.
-	 * @throws HyracksDataException
-	 *             If the BufferCache throws while un/pinning or un/latching.
-	 * @throws TreeIndexException
-	 *             If there is no matching tuple in the index.
-	 */
-	public void update(ITupleReference tuple, IIndexOpContext ictx)
-			throws HyracksDataException, TreeIndexException;
-
-	/**
-	 * Deletes the tuple in the index matching the given tuple.
-	 * 
-	 * @param tuple
-	 *            Tuple to be deleted.
-	 * @param ictx
-	 *            Existing operation context.
-	 * @throws HyracksDataException
-	 *             If the BufferCache throws while un/pinning or un/latching.
-	 * @throws TreeIndexException
-	 *             If there is no matching tuple in the index.
-	 */
-	public void delete(ITupleReference tuple, IIndexOpContext ictx)
-			throws HyracksDataException, TreeIndexException;
+	public ITreeIndexAccessor createAccessor();
 
 	/**
 	 * Prepares the index for bulk loading, returning a bulk load context. The
@@ -125,10 +70,12 @@
 	 *             If the BufferCache throws while un/pinning or un/latching.
 	 * @throws TreeIndexException
 	 *             If the tree is not empty.
+	 * @throws PageAllocationException
 	 * @returns A new context for bulk loading, required for appending tuples.
 	 */
-	public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException,
-			HyracksDataException;
+	public IIndexBulkLoadContext beginBulkLoad(float fillFactor)
+			throws TreeIndexException, HyracksDataException,
+			PageAllocationException;
 
 	/**
 	 * Append a tuple to the index in the context of a bulk load.
@@ -139,9 +86,11 @@
 	 *            Existing bulk load context.
 	 * @throws HyracksDataException
 	 *             If the BufferCache throws while un/pinning or un/latching.
+	 * @throws PageAllocationException
 	 */
 	public void bulkLoadAddTuple(ITupleReference tuple,
-			IIndexBulkLoadContext ictx) throws HyracksDataException;
+			IIndexBulkLoadContext ictx) throws HyracksDataException,
+			PageAllocationException;
 
 	/**
 	 * Finalize the bulk loading operation in the given context.
@@ -150,24 +99,12 @@
 	 *            Existing bulk load context to be finalized.
 	 * @throws HyracksDataException
 	 *             If the BufferCache throws while un/pinning or un/latching.
+	 * @throws PageAllocationException
 	 */
 	public void endBulkLoad(IIndexBulkLoadContext ictx)
-			throws HyracksDataException;
+			throws HyracksDataException, PageAllocationException;
 
 	/**
-	 * Open the given cursor for a disk-order scan, positioning the cursor to
-	 * the first leaf tuple.
-	 * 
-	 * @param icursor
-	 *            Cursor to be opened for disk-order scanning.
-	 * @param ictx
-	 *            Existing operation context.
-	 * @throws HyracksDataException
-	 *             If the BufferCache throws while un/pinning or un/latching.
-	 */
-	public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException;
-	
-	/**
 	 * @return The index's leaf frame factory.
 	 */
 	public ITreeIndexFrameFactory getLeafFrameFactory();
@@ -176,7 +113,7 @@
 	 * @return The index's interior frame factory.
 	 */
 	public ITreeIndexFrameFactory getInteriorFrameFactory();
-	
+
 	/**
 	 * @return The index's free page manager.
 	 */
@@ -186,7 +123,7 @@
 	 * @return The number of fields tuples of this index have.
 	 */
 	public int getFieldCount();
-	
+
 	/**
 	 * @return The current root page id of this index.
 	 */
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java
new file mode 100644
index 0000000..1e679b2
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java
@@ -0,0 +1,101 @@
+/*
+ * 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.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Client handle for performing operations
+ * (insert/delete/update/search/diskorderscan) on an ITreeIndex. An
+ * ITreeIndexAccessor is not thread safe, but different ITreeIndexAccessors can
+ * concurrently operate on the same ITreeIndex (i.e., the ITreeIndex must allow
+ * concurrent operations).
+ */
+public interface ITreeIndexAccessor {
+	/**
+	 * Inserts the given tuple.
+	 * 
+	 * @param tuple
+	 *            Tuple to be inserted.
+	 * @throws HyracksDataException
+	 *             If the BufferCache throws while un/pinning or un/latching.
+	 * @throws TreeIndexException
+	 *             If an index-specific constraint is violated, e.g., the key
+	 *             already exists.
+	 * @throws PageAllocationException
+	 */
+	public void insert(ITupleReference tuple) throws HyracksDataException,
+			TreeIndexException, PageAllocationException;
+
+	/**
+	 * Updates the tuple in the index matching the given tuple with the new
+	 * contents in the given tuple.
+	 * 
+	 * @param tuple
+	 *            Tuple whose match in the index is to be update with the given
+	 *            tuples contents.
+	 * @throws HyracksDataException
+	 *             If the BufferCache throws while un/pinning or un/latching.
+	 * @throws TreeIndexException
+	 *             If there is no matching tuple in the index.
+	 * @throws PageAllocationException
+	 */
+	public void update(ITupleReference tuple) throws HyracksDataException,
+			TreeIndexException, PageAllocationException;
+
+	/**
+	 * Deletes the tuple in the index matching the given tuple.
+	 * 
+	 * @param tuple
+	 *            Tuple to be deleted.
+	 * @throws HyracksDataException
+	 *             If the BufferCache throws while un/pinning or un/latching.
+	 * @throws TreeIndexException
+	 *             If there is no matching tuple in the index.
+	 * @throws PageAllocationException
+	 */
+	public void delete(ITupleReference tuple) throws HyracksDataException,
+			TreeIndexException, PageAllocationException;
+
+	/**
+	 * Open the given cursor for an index search using the given predicate as
+	 * search condition.
+	 * 
+	 * @param icursor
+	 *            Cursor over the index entries satisfying searchPred.
+	 * @param searchPred
+	 *            Search condition.
+	 * @throws HyracksDataException
+	 *             If the BufferCache throws while un/pinning or un/latching.
+	 * @throws TreeIndexException
+	 * @throws PageAllocationException
+	 */
+	public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred)
+			throws HyracksDataException, TreeIndexException, PageAllocationException;
+
+	/**
+	 * Open the given cursor for a disk-order scan, positioning the cursor to
+	 * the first leaf tuple.
+	 * 
+	 * @param icursor
+	 *            Cursor to be opened for disk-order scanning.
+	 * @throws HyracksDataException
+	 *             If the BufferCache throws while un/pinning or un/latching.
+	 */
+	public void diskOrderScan(ITreeIndexCursor cursor)
+			throws HyracksDataException;
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
index 56dfd47..fdfedb2 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
@@ -15,6 +15,7 @@
 
 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.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
@@ -27,7 +28,7 @@
 	public void next() throws Exception;
 
 	public void open(ICursorInitialState initialState,
-			ISearchPredicate searchPred) throws Exception;
+			ISearchPredicate searchPred) throws HyracksDataException;
 
 	public ICachedPage getPage();
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/PageAllocationException.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/PageAllocationException.java
new file mode 100644
index 0000000..e6eec66
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/PageAllocationException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.api;
+
+public class PageAllocationException extends Exception {
+
+	private static final long serialVersionUID = 1L;
+
+	public PageAllocationException(Throwable cause) {
+        super(cause);
+    }
+    
+    public PageAllocationException(String message) {
+        super(message);
+    }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java
index df372f4c..5f3b0b3 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java
@@ -16,26 +16,15 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.util.HashMap;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 
 public class IndexRegistry<IndexType> {
 
 	private HashMap<Integer, IndexType> map = new HashMap<Integer, IndexType>();
-	private Lock registryLock = new ReentrantLock();
 
 	public IndexType get(int fileId) {
 		return map.get(fileId);
 	}
 
-	public void lock() {
-		registryLock.lock();
-	}
-
-	public void unlock() {
-		registryLock.unlock();
-	}
-
 	public void register(int fileId, IndexType index) {
 		map.put(fileId, index);
 	}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
index 63edc06..8d5582d 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
@@ -23,6 +23,7 @@
 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.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
 
 public class TreeIndexBulkLoadOperatorNodePushable extends
 		AbstractUnaryInputSinkOperatorNodePushable {
@@ -73,15 +74,21 @@
 		int tupleCount = accessor.getTupleCount();
 		for (int i = 0; i < tupleCount; i++) {
 			tuple.reset(accessor, i);
-			treeIndexOpHelper.getTreeIndex().bulkLoadAddTuple(tuple,
-					bulkLoadCtx);
+			try {
+				treeIndexOpHelper.getTreeIndex().bulkLoadAddTuple(tuple,
+						bulkLoadCtx);
+			} catch (PageAllocationException e) {
+				throw new HyracksDataException(e);
+			}
 		}
 	}
 
 	@Override
 	public void close() throws HyracksDataException {
 		try {
-			treeIndexOpHelper.getTreeIndex().endBulkLoad(bulkLoadCtx);
+			treeIndexOpHelper.getTreeIndex().endBulkLoad(bulkLoadCtx);		
+		} catch (PageAllocationException e) {
+			throw new HyracksDataException(e);
 		} finally {
 			treeIndexOpHelper.deinit();
 		}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index ab7b335..184bb4e 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -24,10 +24,9 @@
 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.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 
 public class TreeIndexDiskOrderScanOperatorNodePushable extends
 		AbstractUnaryOutputSourceOperatorNodePushable {
@@ -43,19 +42,16 @@
 
 	@Override
 	public void initialize() throws HyracksDataException {
-
 		ITreeIndexFrame cursorFrame = treeIndexOpHelper.getOperatorDescriptor()
 				.getTreeIndexLeafFactory().createFrame();
 		ITreeIndexCursor cursor = treeIndexOpHelper
 				.createDiskOrderScanCursor(cursorFrame);
-		IIndexOpContext diskOrderScanOpCtx = treeIndexOpHelper.getTreeIndex()
-				.createOpContext(IndexOp.DISKORDERSCAN);
+		ITreeIndexAccessor indexAccessor = treeIndexOpHelper.getTreeIndex().createAccessor();
 		try {
-
 			treeIndexOpHelper.init();
 			writer.open();
 			try {
-				treeIndexOpHelper.getTreeIndex().diskOrderScan(cursor, diskOrderScanOpCtx);
+				indexAccessor.diskOrderScan(cursor);
 
 				int fieldCount = treeIndexOpHelper.getTreeIndex()
 						.getFieldCount();
@@ -90,7 +86,6 @@
 						}
 					}
 				}
-
 				if (appender.getTupleCount() > 0) {
 					FrameUtils.flushFrame(frame, writer);
 				}
@@ -101,7 +96,6 @@
 				cursor.close();
 				writer.close();
 			}
-
 		} catch (Exception e) {
 			deinitialize();
 			throw new HyracksDataException(e);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
index 6614fba..65b799e 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
@@ -30,87 +30,75 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
-public class TreeIndexDropOperatorNodePushable extends
-		AbstractOperatorNodePushable {
-	private static final Logger LOGGER = Logger
-			.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
+public class TreeIndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
+    private static final Logger LOGGER = Logger.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
 
-	private final IHyracksTaskContext ctx;
-	private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
-	private IStorageManagerInterface storageManager;
-	private IFileSplitProvider fileSplitProvider;
-	private int partition;
+    private final IHyracksTaskContext ctx;
+    private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
+    private IStorageManagerInterface storageManager;
+    private IFileSplitProvider fileSplitProvider;
+    private int partition;
 
-	public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx,
-			IStorageManagerInterface storageManager,
-			IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
-			IFileSplitProvider fileSplitProvider, int partition) {
-		this.ctx = ctx;
-		this.storageManager = storageManager;
-		this.treeIndexRegistryProvider = treeIndexRegistryProvider;
-		this.fileSplitProvider = fileSplitProvider;
-		this.partition = partition;
-	}
+    public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx, IStorageManagerInterface storageManager,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
+            int partition) {
+        this.ctx = ctx;
+        this.storageManager = storageManager;
+        this.treeIndexRegistryProvider = treeIndexRegistryProvider;
+        this.fileSplitProvider = fileSplitProvider;
+        this.partition = partition;
+    }
 
-	@Override
-	public void deinitialize() throws HyracksDataException {
-	}
+    @Override
+    public void deinitialize() throws HyracksDataException {
+    }
 
-	@Override
-	public int getInputArity() {
-		return 0;
-	}
+    @Override
+    public int getInputArity() {
+        return 0;
+    }
 
-	@Override
-	public IFrameWriter getInputFrameWriter(int index) {
-		return null;
-	}
+    @Override
+    public IFrameWriter getInputFrameWriter(int index) {
+        return null;
+    }
 
-	@Override
-	public void initialize() throws HyracksDataException {
-		try {
+    @Override
+    public void initialize() throws HyracksDataException {
+        try {
+            IndexRegistry<ITreeIndex> treeIndexRegistry = treeIndexRegistryProvider.getRegistry(ctx);
+            IBufferCache bufferCache = storageManager.getBufferCache(ctx);
+            IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
+            
+            FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
+            int indexFileId = -1;
+            synchronized (fileMapProvider) {
+                boolean fileIsMapped = fileMapProvider.isMapped(f);
+                if (!fileIsMapped) {
+                    throw new HyracksDataException("Cannot drop Tree with name " + f.toString()
+                            + ". No file mapping exists.");
+                }
+                indexFileId = fileMapProvider.lookupFileId(f);
+            }
+            // Unregister tree instance.
+            synchronized (treeIndexRegistry) {
+                treeIndexRegistry.unregister(indexFileId);
+            }
 
-			IndexRegistry<ITreeIndex> treeIndexRegistry = treeIndexRegistryProvider
-					.getRegistry(ctx);
-			IBufferCache bufferCache = storageManager.getBufferCache(ctx);
-			IFileMapProvider fileMapProvider = storageManager
-					.getFileMapProvider(ctx);
+            // remove name to id mapping
+            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("Tree Drop Operator Failed Due To Exception: " + e.getMessage());
+            }
+        }
+    }
 
-			FileReference f = fileSplitProvider.getFileSplits()[partition]
-					.getLocalFile();
-
-			boolean fileIsMapped = fileMapProvider.isMapped(f);
-			if (!fileIsMapped) {
-				throw new HyracksDataException("Cannot drop Tree with name "
-						+ f.toString() + ". No file mapping exists.");
-			}
-
-			int indexFileId = fileMapProvider.lookupFileId(f);
-
-			// unregister tree instance
-			treeIndexRegistry.lock();
-			try {
-				treeIndexRegistry.unregister(indexFileId);
-			} finally {
-				treeIndexRegistry.unlock();
-			}
-
-			// remove name to id mapping
-			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("Tree Drop Operator Failed Due To Exception: "
-						+ e.getMessage());
-			}
-		}
-	}
-
-	@Override
-	public void setOutputFrameWriter(int index, IFrameWriter writer,
-			RecordDescriptor recordDesc) {
-	}
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+    }
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index 0ae513b..ba4106b 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -23,110 +23,103 @@
 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.common.api.IIndexOpContext;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 
-public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends
-		AbstractUnaryInputUnaryOutputOperatorNodePushable {
-	private final TreeIndexOpHelper treeIndexOpHelper;
-	private FrameTupleAccessor accessor;
-	private final IRecordDescriptorProvider recordDescProvider;
-	private final IndexOp op;
-	private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
-	private ByteBuffer writeBuffer;
-	private IIndexOpContext opCtx;
+public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    private final TreeIndexOpHelper treeIndexOpHelper;
+    private FrameTupleAccessor accessor;
+    private final IRecordDescriptorProvider recordDescProvider;
+    private final IndexOp op;
+    private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+    private ByteBuffer writeBuffer;
+    private ITreeIndexAccessor indexAccessor;
 
-	public TreeIndexInsertUpdateDeleteOperatorNodePushable(
-			AbstractTreeIndexOperatorDescriptor opDesc,
-			IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
-			IRecordDescriptorProvider recordDescProvider, IndexOp op) {
-		treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory()
-				.createTreeIndexOpHelper(opDesc, ctx, partition,
-						IndexHelperOpenMode.OPEN);
-		this.recordDescProvider = recordDescProvider;
-		this.op = op;
-		tuple.setFieldPermutation(fieldPermutation);
-	}
+    public TreeIndexInsertUpdateDeleteOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
+            IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
+            IRecordDescriptorProvider recordDescProvider, IndexOp op) {
+        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
+                IndexHelperOpenMode.OPEN);
+        this.recordDescProvider = recordDescProvider;
+        this.op = op;
+        tuple.setFieldPermutation(fieldPermutation);
+    }
 
-	@Override
-	public void open() throws HyracksDataException {
-		AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
-				.getOperatorDescriptor();
-		RecordDescriptor inputRecDesc = recordDescProvider
-				.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-		accessor = new FrameTupleAccessor(treeIndexOpHelper
-				.getHyracksTaskContext().getFrameSize(), inputRecDesc);
-		writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
-		writer.open();
-		try {
-			treeIndexOpHelper.init();
-			treeIndexOpHelper.getTreeIndex().open(
-					treeIndexOpHelper.getIndexFileId());
-			opCtx = treeIndexOpHelper.getTreeIndex().createOpContext(op);
-		} catch (Exception e) {
-			// cleanup in case of failure
-			treeIndexOpHelper.deinit();
-			throw new HyracksDataException(e);
-		}
-	}
+    @Override
+    public void open() throws HyracksDataException {
+        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
+                .getOperatorDescriptor();
+        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
+        writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+        writer.open();
+        try {
+            treeIndexOpHelper.init();
+            treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
+            indexAccessor = treeIndexOpHelper.getTreeIndex().createAccessor();
+        } catch (Exception e) {
+            // cleanup in case of failure
+            treeIndexOpHelper.deinit();
+            throw new HyracksDataException(e);
+        }
+    }
 
-	@Override
-	public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-		final ITreeIndex treeIndex = treeIndexOpHelper.getTreeIndex();
-		accessor.reset(buffer);
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        final ITreeIndex treeIndex = treeIndexOpHelper.getTreeIndex();
+        accessor.reset(buffer);
 
-		int tupleCount = accessor.getTupleCount();
-		for (int i = 0; i < tupleCount; i++) {
-			tuple.reset(accessor, i);
-			try {
-				switch (op) {
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            tuple.reset(accessor, i);
+            try {
+                switch (op) {
 
-				case INSERT: {
-					treeIndex.insert(tuple, opCtx);
-					break;
-				}
+                    case INSERT: {
+                        indexAccessor.insert(tuple);
+                        break;
+                    }
 
-				case UPDATE: {
-					treeIndex.update(tuple, opCtx);
-					break;
-				}
-				
-				case DELETE: {
-					treeIndex.delete(tuple, opCtx);
-					break;
-				}
-					
-				default: {
-					throw new HyracksDataException("Unsupported operation "
-							+ op + " in tree index InsertUpdateDelete operator");
-				}
+                    case UPDATE: {
+                        indexAccessor.update(tuple);
+                        break;
+                    }
 
-				}
-			} catch (HyracksDataException e) {
-				throw e;
-			} catch (Exception e) {
-				throw new HyracksDataException(e);
-			}
-		}
+                    case DELETE: {
+                        indexAccessor.delete(tuple);
+                        break;
+                    }
 
-		// pass a copy of the frame to next op
-		System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0,
-				buffer.capacity());
-		FrameUtils.flushFrame(writeBuffer, writer);
-	}
+                    default: {
+                        throw new HyracksDataException("Unsupported operation " + op
+                                + " in tree index InsertUpdateDelete operator");
+                    }
 
-	@Override
-	public void close() throws HyracksDataException {
-		try {
-			writer.close();
-		} finally {
-			treeIndexOpHelper.deinit();
-		}
-	}
+                }
+            } catch (HyracksDataException e) {
+                throw e;
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
 
-	@Override
-	public void fail() throws HyracksDataException {
-		writer.fail();
-	}
+        // pass a copy of the frame to next op
+        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
+        FrameUtils.flushFrame(writeBuffer, writer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            writer.close();
+        } finally {
+            treeIndexOpHelper.deinit();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
 }
\ No newline at end of file
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
index 6fea7e5..ff2f043 100644
--- 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
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 
@@ -56,52 +57,42 @@
         IFileSplitProvider fileSplitProvider = opDesc.getTreeIndexFileSplitProvider();
 
         FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-        boolean fileIsMapped = fileMapProvider.isMapped(f);
-        if (!fileIsMapped) {
-            bufferCache.createFile(f);
-        }
-        int fileId = fileMapProvider.lookupFileId(f);
-        try {
-            bufferCache.openFile(fileId);
-        } catch (HyracksDataException e) {
-            // Revert state of buffer cache since file failed to open.
+        int fileId = -1;
+        synchronized (fileMapProvider) {
+            boolean fileIsMapped = fileMapProvider.isMapped(f);
             if (!fileIsMapped) {
-                bufferCache.deleteFile(fileId);
+                bufferCache.createFile(f);
             }
-            throw e;
+            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;
-
         IndexRegistry<ITreeIndex> treeIndexRegistry = opDesc.getTreeIndexRegistryProvider().getRegistry(ctx);
         // Create new tree and register it.
-        treeIndexRegistry.lock();
-        try {
+        synchronized (treeIndexRegistry) {
             // Check if tree has already been registered by another thread.
             treeIndex = treeIndexRegistry.get(indexFileId);
             if (treeIndex != null) {
                 return;
             }
-            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(comparators);
+            cmp = IndexUtils.createMultiComparator(opDesc.getTreeIndexComparatorFactories());
             treeIndex = createTreeIndex();
             if (mode == IndexHelperOpenMode.CREATE) {
-                try {
-                    treeIndex.create(indexFileId);
-                } catch (Exception e) {
-                	e.printStackTrace();
-                    throw new HyracksDataException(e);
-                }
+                treeIndex.create(indexFileId);
             }
             treeIndex.open(indexFileId);
             treeIndexRegistry.register(indexFileId, treeIndex);
-        } finally {
-            treeIndexRegistry.unlock();
         }
     }
 
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexUtils.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexUtils.java
new file mode 100644
index 0000000..389855f
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexUtils.java
@@ -0,0 +1,29 @@
+/*
+ * 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.util;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public class IndexUtils {
+	public static MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+    	IBinaryComparator[] cmps = new IBinaryComparator[cmpFactories.length];
+    	for (int i = 0; i < cmpFactories.length; i++) {
+    		cmps[i] = cmpFactories[i].createBinaryComparator(); 
+    	}
+    	return new MultiComparator(cmps);
+    }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java
index b3afe4a..5085e31 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java
@@ -23,11 +23,11 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 
 public interface IInvertedIndexOperatorDescriptorHelper extends ITreeIndexOperatorDescriptorHelper {
-    public IFileSplitProvider getInvIndexFileSplitProvider();
+    public IFileSplitProvider getInvListsFileSplitProvider();
 
-    public IBinaryComparatorFactory[] getInvIndexComparatorFactories();
+    public IBinaryComparatorFactory[] getInvListsComparatorFactories();
 
-    public ITypeTrait[] getInvIndexTypeTraits();
+    public ITypeTrait[] getInvListsTypeTraits();
 
     public IIndexRegistryProvider<InvertedIndex> getInvIndexRegistryProvider();
 }
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java
new file mode 100644
index 0000000..92770d6
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java
@@ -0,0 +1,22 @@
+/*
+ * 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.invertedindex.api;
+
+import java.io.Serializable;
+
+public interface IInvertedIndexSearchModifierFactory extends Serializable {
+    public IInvertedIndexSearchModifier createSearchModifier();
+}
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 b93f064..24d8808 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,7 +21,6 @@
 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.IPrimitiveValueProviderFactory;
 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;
@@ -41,26 +40,26 @@
     // btree
     protected final IFileSplitProvider btreeFileSplitProvider;
     protected final IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
-    protected final ITreeIndexFrameFactory interiorFrameFactory;
-    protected final ITreeIndexFrameFactory leafFrameFactory;
+    protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
+    protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
     protected final ITypeTrait[] btreeTypeTraits;
     protected final IBinaryComparatorFactory[] btreeComparatorFactories;
-    protected final ITreeIndexOpHelperFactory opHelperFactory;
+    protected final ITreeIndexOpHelperFactory btreeOpHelperFactory;
 
     // inverted index
-    protected final IFileSplitProvider invIndexFileSplitProvider;
+    protected final IFileSplitProvider invListsFileSplitProvider;
     protected final IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider;
-    protected final ITypeTrait[] invIndexTypeTraits;
-    protected final IBinaryComparatorFactory[] invIndexComparatorFactories;
+    protected final ITypeTrait[] invListsTypeTraits;
+    protected final IBinaryComparatorFactory[] invListComparatorFactories;
 
     public AbstractInvertedIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
             RecordDescriptor recDesc, IStorageManagerInterface storageManager,
             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) {
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories,
+            ITreeIndexOpHelperFactory btreeOpHelperFactory, IFileSplitProvider invListsFileSplitProvider,
+            IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories) {
         super(spec, inputArity, outputArity);
 
         // general
@@ -69,20 +68,21 @@
         // btree
         this.btreeFileSplitProvider = btreeFileSplitProvider;
         this.treeIndexRegistryProvider = treeIndexRegistryProvider;
-        this.interiorFrameFactory = interiorFrameFactory;
-        this.leafFrameFactory = leafFrameFactory;
+        this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
+        this.btreeLeafFrameFactory = btreeLeafFrameFactory;
         this.btreeTypeTraits = btreeTypeTraits;
         this.btreeComparatorFactories = btreeComparatorFactories;
-        this.opHelperFactory = opHelperFactory;
+        this.btreeOpHelperFactory = btreeOpHelperFactory;
 
         // inverted index
-        this.invIndexFileSplitProvider = invIndexFileSplitProvider;
+        this.invListsFileSplitProvider = invListsFileSplitProvider;
         this.invIndexRegistryProvider = invIndexRegistryProvider;
-        this.invIndexTypeTraits = invIndexTypeTraits;
-        this.invIndexComparatorFactories = invIndexComparatorFactories;
+        this.invListsTypeTraits = invListsTypeTraits;
+        this.invListComparatorFactories = invListComparatorFactories;
 
-        if (outputArity > 0)
+        if (outputArity > 0) {
             recordDescriptors[0] = recDesc;
+        }
     }
 
     @Override
@@ -107,12 +107,12 @@
 
     @Override
     public ITreeIndexFrameFactory getTreeIndexInteriorFactory() {
-        return interiorFrameFactory;
+        return btreeInteriorFrameFactory;
     }
 
     @Override
     public ITreeIndexFrameFactory getTreeIndexLeafFactory() {
-        return leafFrameFactory;
+        return btreeLeafFrameFactory;
     }
 
     @Override
@@ -136,22 +136,22 @@
     }
 
     @Override
-    public IBinaryComparatorFactory[] getInvIndexComparatorFactories() {
-        return invIndexComparatorFactories;
+    public IBinaryComparatorFactory[] getInvListsComparatorFactories() {
+        return invListComparatorFactories;
     }
 
     @Override
-    public IFileSplitProvider getInvIndexFileSplitProvider() {
-        return invIndexFileSplitProvider;
+    public IFileSplitProvider getInvListsFileSplitProvider() {
+        return invListsFileSplitProvider;
     }
 
     @Override
-    public ITypeTrait[] getInvIndexTypeTraits() {
-        return invIndexTypeTraits;
+    public ITypeTrait[] getInvListsTypeTraits() {
+        return invListsTypeTraits;
     }
 
     @Override
     public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory() {
-        return opHelperFactory;
+        return btreeOpHelperFactory;
     }
 }
\ No newline at end of file
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index b1cdefc..2d7aa1c 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -31,17 +31,17 @@
     private final IBinaryTokenizerFactory tokenizerFactory;
     // fields that will be tokenized
     private final int[] tokenFields;
-    // operator will emit these projected fields for each token, e.g., as
+    // operator will append these key fields to each token, e.g., as
     // payload for an inverted list
-    // WARNING: too many projected fields can cause significant data blowup
-    private final int[] projFields;
+    // WARNING: too many key fields can cause significant data blowup
+    private final int[] keyFields;
 
     public BinaryTokenizerOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IBinaryTokenizerFactory tokenizerFactory, int[] tokenFields, int[] projFields) {
+            IBinaryTokenizerFactory tokenizerFactory, int[] tokenFields, int[] keyFields) {
         super(spec, 1, 1);
         this.tokenizerFactory = tokenizerFactory;
         this.tokenFields = tokenFields;
-        this.projFields = projFields;
+        this.keyFields = keyFields;
         recordDescriptors[0] = recDesc;
     }
 
@@ -49,6 +49,6 @@
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
             int partition, int nPartitions) throws HyracksDataException {
         return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(odId, 0),
-                recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, projFields);
+                recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, keyFields);
     }
 }
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 28e3db8..96e0159 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
@@ -26,7 +26,6 @@
 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;
 
@@ -36,29 +35,27 @@
 
     private final int[] fieldPermutation;
     private final float btreeFillFactor;
-    private final IInvertedListBuilder invListBuilder;
 
     public InvertedIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
             int[] fieldPermutation, IFileSplitProvider btreeFileSplitProvider,
-            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, ITreeIndexFrameFactory interiorFrameFactory,
-            ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] btreeTypeTraits,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+            ITreeIndexFrameFactory btreeLeafFrameFactory, ITypeTrait[] btreeTypeTraits,
             IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
-            ITreeIndexOpHelperFactory opHelperFactory, IFileSplitProvider invIndexFileSplitProvider,
-            IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
-            IBinaryComparatorFactory[] invIndexComparatorFactories, IInvertedListBuilder invListBuilder) {
+            ITreeIndexOpHelperFactory btreeOpHelperFactory, IFileSplitProvider invListsFileSplitProvider,
+            IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListComparatorFactories) {
         super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, treeIndexRegistryProvider,
-                interiorFrameFactory, leafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeFillFactor,
-                opHelperFactory, invIndexFileSplitProvider, invIndexRegistryProvider, invIndexTypeTraits,
-                invIndexComparatorFactories);
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories, 
+                btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider, invListsTypeTraits,
+                invListComparatorFactories);
         this.fieldPermutation = fieldPermutation;
         this.btreeFillFactor = btreeFillFactor;
-        this.invListBuilder = invListBuilder;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, btreeFillFactor,
-                invListBuilder, recordDescProvider);
+                recordDescProvider);
     }
 }
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 9d421b0..b0f62c6 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,16 @@
 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.common.api.PageAllocationException;
 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.FixedSizeElementInvertedListBuilder;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 
 public class InvertedIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private final TreeIndexOpHelper treeIndexOpHelper;
+    private final TreeIndexOpHelper btreeOpHelper;
     private float btreeFillFactor;
 
     private final InvertedIndexOpHelper invIndexOpHelper;
@@ -45,44 +47,52 @@
 
     public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
             IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
-            IInvertedListBuilder invListBuilder, IRecordDescriptorProvider recordDescProvider) {
-        treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
+            IRecordDescriptorProvider recordDescProvider) {
+        btreeOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
                 IndexHelperOpenMode.CREATE);
-        invIndexOpHelper = new InvertedIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
+        invIndexOpHelper = new InvertedIndexOpHelper(btreeOpHelper, opDesc, ctx, partition);
         this.btreeFillFactor = btreeFillFactor;
         this.recordDescProvider = recordDescProvider;
         this.ctx = ctx;
-        this.invListBuilder = invListBuilder;
+        this.invListBuilder = new FixedSizeElementInvertedListBuilder(opDesc.getInvListsTypeTraits());
         tuple.setFieldPermutation(fieldPermutation);
     }
 
     @Override
     public void open() throws HyracksDataException {
-        AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) treeIndexOpHelper
+        AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) btreeOpHelper
                 .getOperatorDescriptor();
         RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
 
-        // btree
+        // BTree.
         try {
-            treeIndexOpHelper.init();
-            treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
+            btreeOpHelper.init();
+            btreeOpHelper.getTreeIndex().open(btreeOpHelper.getIndexFileId());
         } catch (Exception e) {
-            // cleanup in case of failure
-            treeIndexOpHelper.deinit();
-            throw new HyracksDataException(e);
+            // Cleanup in case of failure/
+            btreeOpHelper.deinit();
+            if (e instanceof HyracksDataException) {
+                throw (HyracksDataException) e;
+            } else {
+                throw new HyracksDataException(e);
+            }
         }
 
-        // inverted index
+        // Inverted Index.
         try {
             invIndexOpHelper.init();
             invIndexOpHelper.getInvIndex().open(invIndexOpHelper.getInvIndexFileId());
             bulkLoadCtx = invIndexOpHelper.getInvIndex().beginBulkLoad(invListBuilder, ctx.getFrameSize(),
                     btreeFillFactor);
         } catch (Exception e) {
-            // cleanup in case of failure
+            // Cleanup in case of failure.
             invIndexOpHelper.deinit();
-            throw new HyracksDataException(e);
+            if (e instanceof HyracksDataException) {
+                throw (HyracksDataException) e;
+            } else {
+                throw new HyracksDataException(e);
+            }
         }
     }
 
@@ -92,7 +102,11 @@
         int tupleCount = accessor.getTupleCount();
         for (int i = 0; i < tupleCount; i++) {
             tuple.reset(accessor, i);
-            invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
+            try {
+                invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
+            } catch (PageAllocationException e) {
+                throw new HyracksDataException(e);
+            }
         }
     }
 
@@ -100,8 +114,14 @@
     public void close() throws HyracksDataException {
         try {
             invIndexOpHelper.getInvIndex().endBulkLoad(bulkLoadCtx);
+        } catch (PageAllocationException e) {
+            throw new HyracksDataException(e);
         } finally {
-            treeIndexOpHelper.deinit();
+            try {
+                btreeOpHelper.deinit();
+            } finally {
+                invIndexOpHelper.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 684bb20..baa3052 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
@@ -15,16 +15,15 @@
 package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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.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.dataflow.TreeIndexOpHelper;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptorHelper;
 import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -32,6 +31,8 @@
 
 public final class InvertedIndexOpHelper {
 
+    private final TreeIndexOpHelper btreeOpHelper;
+
     private InvertedIndex invIndex;
     private int invIndexFileId = -1;
     private int partition;
@@ -39,97 +40,59 @@
     private IInvertedIndexOperatorDescriptorHelper opDesc;
     private IHyracksTaskContext ctx;
 
-    private IndexHelperOpenMode mode;
-
-    public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
-            int partition, IndexHelperOpenMode mode) {
+    public InvertedIndexOpHelper(TreeIndexOpHelper btreeOpHelper, IInvertedIndexOperatorDescriptorHelper opDesc,
+            final IHyracksTaskContext ctx, int partition) {
+        this.btreeOpHelper = btreeOpHelper;
         this.opDesc = opDesc;
         this.ctx = ctx;
-        this.mode = mode;
         this.partition = partition;
     }
 
+    // TODO: This is very similar to TreeIndexOpHelper. Maybe we can somehow
+    // merge them?
     public void init() throws HyracksDataException {
         IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
         IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
-        IFileSplitProvider fileSplitProvider = opDesc.getInvIndexFileSplitProvider();
+        IFileSplitProvider fileSplitProvider = opDesc.getInvListsFileSplitProvider();
 
         FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
-        boolean fileIsMapped = fileMapProvider.isMapped(f);
-
-        switch (mode) {
-
-            case OPEN: {
-                if (!fileIsMapped) {
-                    throw new HyracksDataException("Trying to open inverted index 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
+        int fileId = -1;
+        synchronized (fileMapProvider) {
+            boolean fileIsMapped = fileMapProvider.isMapped(f);
             if (!fileIsMapped) {
-                bufferCache.deleteFile(fileId);
+                bufferCache.createFile(f);
             }
-            throw e;
+            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
         invIndexFileId = fileId;
         IndexRegistry<InvertedIndex> invIndexRegistry = opDesc.getInvIndexRegistryProvider().getRegistry(ctx);
-        invIndex = invIndexRegistry.get(invIndexFileId);
-        if (invIndex == null) {
-
-            // create new inverted index and register it
-            invIndexRegistry.lock();
-            try {
-                // check if inverted index has already been registered by
-                // another thread
-                invIndex = invIndexRegistry.get(invIndexFileId);
-                if (invIndex == null) {
-                    // this thread should create and register the inverted index
-
-                    IBinaryComparator[] comparators = new IBinaryComparator[opDesc.getInvIndexComparatorFactories().length];
-                    for (int i = 0; i < opDesc.getInvIndexComparatorFactories().length; i++) {
-                        comparators[i] = opDesc.getInvIndexComparatorFactories()[i].createBinaryComparator();
-                    }
-
-                    MultiComparator cmp = new MultiComparator(comparators);
-
-                    // assumes btree has already been registered
-                    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 = (BTree) treeIndexRegistry.get(btreeFileId);
-
-                    invIndex = new InvertedIndex(bufferCache, btree, opDesc.getInvIndexTypeTraits(), cmp);
-                    invIndex.open(invIndexFileId);
-                    invIndexRegistry.register(invIndexFileId, invIndex);
-                }
-            } finally {
-                invIndexRegistry.unlock();
+        // create new inverted index and register it
+        synchronized (invIndexRegistry) {
+            // check if inverted index has already been registered by
+            // another thread
+            invIndex = invIndexRegistry.get(invIndexFileId);
+            if (invIndex != null) {
+                return;
             }
+            // Create and register the inverted index.
+            MultiComparator cmp = IndexUtils.createMultiComparator(opDesc.getInvListsComparatorFactories());
+            // Assumes btreeOpHelper.init() has already been called.
+            BTree btree = (BTree) btreeOpHelper.getTreeIndex();
+            invIndex = new InvertedIndex(bufferCache, btree, opDesc.getInvListsTypeTraits(), cmp);
+            invIndex.open(invIndexFileId);
+            invIndexRegistry.register(invIndexFileId, invIndex);
         }
     }
 
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
new file mode 100644
index 0000000..fe0c255
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
@@ -0,0 +1,58 @@
+package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class InvertedIndexSearchOperatorDescriptor extends AbstractInvertedIndexOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final int queryField;
+    private final IBinaryTokenizerFactory queryTokenizerFactory;
+    private final IInvertedIndexSearchModifierFactory searchModifierFactory;
+
+    public InvertedIndexSearchOperatorDescriptor(JobSpecification spec, int queryField,
+            IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
+            IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
+            ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories,
+            ITreeIndexOpHelperFactory btreeOpHelperFactory, IFileSplitProvider invListsFileSplitProvider,
+            IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
+            IBinaryComparatorFactory[] invListsComparatorFactories,
+            IInvertedIndexSearchModifierFactory searchModifierFactory, IBinaryTokenizerFactory queryTokenizerFactory,
+            RecordDescriptor recDesc) {
+        super(spec, 1, 1, recDesc, storageManager, btreeFileSplitProvider, treeIndexRegistryProvider,
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories,
+                btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider, invListsTypeTraits,
+                invListsComparatorFactories);
+        this.queryField = queryField;
+        this.searchModifierFactory = searchModifierFactory;
+        this.queryTokenizerFactory = queryTokenizerFactory;
+    }
+
+	@Override
+	public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+			IRecordDescriptorProvider recordDescProvider, int partition,
+			int nPartitions) throws HyracksDataException {
+		IBinaryTokenizer tokenizer = queryTokenizerFactory.createTokenizer();
+        IInvertedIndexSearchModifier searchModifier = searchModifierFactory.createSearchModifier();
+        return new InvertedIndexSearchOperatorNodePushable(this, ctx, partition, queryField, searchModifier, tokenizer,
+                recordDescProvider);
+	}
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
new file mode 100644
index 0000000..308ea3b
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
@@ -0,0 +1,168 @@
+package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+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.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
+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.impls.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.SearchResultCursor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.TOccurrenceSearcher;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+
+public class InvertedIndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    private final TreeIndexOpHelper btreeOpHelper;
+    private final InvertedIndexOpHelper invIndexOpHelper;
+    private final IHyracksTaskContext ctx;
+    private final int queryField;
+    private FrameTupleAccessor accessor;
+    private FrameTupleReference tuple;
+    private IRecordDescriptorProvider recordDescProvider;
+
+    private final IInvertedIndexSearchModifier searchModifier;
+    private final IBinaryTokenizer queryTokenizer;
+    private TOccurrenceSearcher searcher;
+    private IInvertedIndexResultCursor resultCursor;
+
+    private ByteBuffer writeBuffer;
+    private FrameTupleAppender appender;
+    private ArrayTupleBuilder tb;
+    private DataOutput dos;
+
+    private final AbstractInvertedIndexOperatorDescriptor opDesc;
+
+    public InvertedIndexSearchOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
+            IHyracksTaskContext ctx, int partition, int queryField, IInvertedIndexSearchModifier searchModifier,
+            IBinaryTokenizer queryTokenizer, IRecordDescriptorProvider recordDescProvider) {
+        this.opDesc = opDesc;
+        btreeOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
+                IndexHelperOpenMode.CREATE);
+        invIndexOpHelper = new InvertedIndexOpHelper(btreeOpHelper, opDesc, ctx, partition);
+        this.ctx = ctx;
+        this.queryField = queryField;
+        this.searchModifier = searchModifier;
+        this.queryTokenizer = queryTokenizer;
+        this.recordDescProvider = recordDescProvider;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+        accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+        tuple = new FrameTupleReference();
+        // BTree.
+        try {
+            btreeOpHelper.init();
+            btreeOpHelper.getTreeIndex().open(btreeOpHelper.getIndexFileId());
+        } catch (Exception e) {
+            // Cleanup in case of failure/
+            btreeOpHelper.deinit();
+            if (e instanceof HyracksDataException) {
+                throw (HyracksDataException) e;
+            } else {
+                throw new HyracksDataException(e);
+            }
+        }
+        // Inverted Index.
+        try {
+            invIndexOpHelper.init();
+            invIndexOpHelper.getInvIndex().open(invIndexOpHelper.getInvIndexFileId());
+        } catch (Exception e) {
+            // Cleanup in case of failure.
+            invIndexOpHelper.deinit();
+            if (e instanceof HyracksDataException) {
+                throw (HyracksDataException) e;
+            } else {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        writeBuffer = btreeOpHelper.getHyracksTaskContext().allocateFrame();
+        tb = new ArrayTupleBuilder(opDesc.getInvListsTypeTraits().length);
+        dos = tb.getDataOutput();
+        appender = new FrameTupleAppender(btreeOpHelper.getHyracksTaskContext().getFrameSize());
+        appender.reset(writeBuffer, true);
+
+        searcher = new TOccurrenceSearcher(ctx, invIndexOpHelper.getInvIndex(), queryTokenizer);
+        resultCursor = new SearchResultCursor(searcher.createResultFrameTupleAccessor(),
+                searcher.createResultTupleReference());
+
+        writer.open();
+    }
+
+    private void writeSearchResults() throws Exception {
+        while (resultCursor.hasNext()) {
+            resultCursor.next();
+            tb.reset();
+            ITupleReference invListElement = resultCursor.getTuple();
+            int invListFields = opDesc.getInvListsTypeTraits().length;
+            for (int i = 0; i < invListFields; i++) {
+                dos.write(invListElement.getFieldData(i), invListElement.getFieldStart(i),
+                        invListElement.getFieldLength(i));
+                tb.addFieldEndOffset();
+            }
+            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                FrameUtils.flushFrame(writeBuffer, writer);
+                appender.reset(writeBuffer, true);
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    throw new IllegalStateException();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        try {
+            for (int i = 0; i < tupleCount; i++) {
+                tuple.reset(accessor, i);
+                searcher.reset();
+                try {
+                    searcher.reset();
+                    searcher.search(resultCursor, tuple, queryField, searchModifier);
+                    writeSearchResults();
+                } catch (OccurrenceThresholdPanicException e) {
+                    // Ignore panic cases for now.
+                }
+            }
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        try {
+            if (appender.getTupleCount() > 0) {
+                FrameUtils.flushFrame(writeBuffer, writer);
+            }
+            writer.close();
+        } finally {
+            try {
+                btreeOpHelper.deinit();
+            } finally {
+                invIndexOpHelper.deinit();
+            }
+        }
+    }
+}
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 75fdf2a..f6ccbb3 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
@@ -29,10 +29,11 @@
 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.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.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -78,7 +79,7 @@
     }
 
     public BulkLoadContext beginBulkLoad(IInvertedListBuilder invListBuilder, int hyracksFrameSize,
-            float btreeFillFactor) throws HyracksDataException, TreeIndexException {
+            float btreeFillFactor) throws HyracksDataException, TreeIndexException, PageAllocationException {
         BulkLoadContext ctx = new BulkLoadContext(invListBuilder, hyracksFrameSize, btreeFillFactor);
         ctx.init(rootPageId, fileId);
         return ctx;
@@ -90,7 +91,7 @@
     // the next invListCmp.getKeyFieldCount() fields in tuple are keys of the
     // inverted list (e.g., primary key)
     // key fields of inverted list are fixed size
-    public void bulkLoadAddTuple(BulkLoadContext ctx, ITupleReference tuple) throws HyracksDataException {
+    public void bulkLoadAddTuple(BulkLoadContext ctx, ITupleReference tuple) throws HyracksDataException, PageAllocationException {
 
         // first inverted list, copy token to baaos and start new list
         if (ctx.currentInvListTokenBaaos.size() == 0) {
@@ -151,9 +152,9 @@
         }
     }
 
-    public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, BTreeOpContext btreeOpCtx,
+    public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, ITreeIndexAccessor btreeAccessor,
             IInvertedListCursor invListCursor) throws Exception {
-        btree.search(btreeCursor, btreePred, btreeOpCtx);
+    	btreeAccessor.search(btreeCursor, btreePred);
 
         boolean ret = false;
         if (btreeCursor.hasNext()) {
@@ -183,7 +184,7 @@
         return ret;
     }
 
-    public void createAndInsertBTreeTuple(BulkLoadContext ctx) throws HyracksDataException {
+    public void createAndInsertBTreeTuple(BulkLoadContext ctx) throws HyracksDataException, PageAllocationException {
         // build tuple
         ctx.btreeTupleBuilder.reset();
         ctx.btreeTupleBuilder.addField(ctx.currentInvListTokenBaaos.getByteArray(), 0,
@@ -204,7 +205,7 @@
         btree.bulkLoadAddTuple(ctx.btreeFrameTupleReference, ctx.btreeBulkLoadCtx);
     }
 
-    public void endBulkLoad(BulkLoadContext ctx) throws HyracksDataException {
+    public void endBulkLoad(BulkLoadContext ctx) throws HyracksDataException, PageAllocationException {
         // create entry in btree for last inverted list
         createAndInsertBTreeTuple(ctx);
         btree.endBulkLoad(ctx.btreeBulkLoadCtx);
@@ -267,7 +268,7 @@
             this.btreeFillFactor = btreeFillFactor;
         }
 
-        public void init(int startPageId, int fileId) throws HyracksDataException, TreeIndexException {
+        public void init(int startPageId, int fileId) throws HyracksDataException, TreeIndexException, PageAllocationException {
             btreeBulkLoadCtx = btree.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR);
             currentPageId = startPageId;
             currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
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 0f88c0b..63efd5f 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
@@ -36,12 +36,11 @@
 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.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.common.api.ITreeIndexAccessor;
 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.invertedindex.api.IInvertedIndexResultCursor;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
@@ -69,7 +68,7 @@
     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;
+    protected final ITreeIndexAccessor btreeAccessor;
 
     protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
             new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
@@ -107,8 +106,6 @@
         invListFieldsWithCount[invListFields.length] = new TypeTrait(4);
         invListKeyLength = tmp;
 
-        btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH);
-
         resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
         resultFrameTupleAcc = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), invListFieldsWithCount);
         resultTuple = new FixedSizeTupleReference(invListFieldsWithCount);
@@ -129,7 +126,8 @@
 
         queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
         queryTokenFrame = ctx.allocateFrame();
-
+        
+        btreeAccessor = invIndex.getBTree().createAccessor();
         currentNumResults = 0;
     }
 
@@ -143,12 +141,12 @@
         currentNumResults = 0;
     }
 
-    public void search(IInvertedIndexResultCursor resultCursor, ITupleReference queryTuple, int queryFieldIndex,
+    public void search(IInvertedIndexResultCursor resultCursor, ITupleReference queryTuple, int queryField,
             IInvertedIndexSearchModifier searchModifier) throws Exception {
 
         queryTokenAppender.reset(queryTokenFrame, true);
-        queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
-                queryTuple.getFieldLength(queryFieldIndex));
+        queryTokenizer.reset(queryTuple.getFieldData(queryField), queryTuple.getFieldStart(queryField),
+                queryTuple.getFieldLength(queryField));
 
         while (queryTokenizer.hasNext()) {
             queryTokenizer.next();
@@ -181,7 +179,7 @@
         invListCursors.clear();
         for (int i = 0; i < numQueryTokens; i++) {
             searchKey.reset(queryTokenAccessor, i);
-            invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursorCache.get(i));
+            invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursorCache.get(i));
             invListCursors.add(invListCursorCache.get(i));
         }
 
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java
new file mode 100644
index 0000000..0db6008
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java
@@ -0,0 +1,28 @@
+/*
+ * 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.invertedindex.searchmodifiers;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class ConjunctiveSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new ConjunctiveSearchModifier();
+    }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java
new file mode 100644
index 0000000..128d9db
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java
@@ -0,0 +1,37 @@
+/*
+ * 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.invertedindex.searchmodifiers;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class EditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int gramLength;
+    private final int edThresh;
+    
+    public EditDistanceSearchModifierFactory(int gramLength, int edThresh) {
+        this.gramLength = gramLength;
+        this.edThresh = edThresh;
+    }
+    
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new EditDistanceSearchModifier(gramLength, edThresh);
+    }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java
new file mode 100644
index 0000000..bd27c03
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.invertedindex.searchmodifiers;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class JaccardSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final float jaccThresh;
+
+    public JaccardSearchModifierFactory(float jaccThresh) {
+        this.jaccThresh = jaccThresh;
+    }
+
+    @Override
+    public IInvertedIndexSearchModifier createSearchModifier() {
+        return new JaccardSearchModifier(jaccThresh);
+    }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
new file mode 100644
index 0000000..db3ad66
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
@@ -0,0 +1,21 @@
+/*
+ * 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.invertedindex.util;
+
+
+public class InvertedIndexUtils {
+	
+}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 4142ebf..daf312f 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -29,18 +29,17 @@
 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.common.api.ITreeIndexAccessor;
 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.rtree.api.IRTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 
@@ -60,7 +59,7 @@
     private ITreeIndexCursor cursor;
     private ITreeIndexFrame interiorFrame;
     private ITreeIndexFrame leafFrame;
-    private RTreeOpContext opCtx;
+    private ITreeIndexAccessor indexAccessor;
 
     private RecordDescriptor recDesc;
 
@@ -110,7 +109,7 @@
                 appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
                 appender.reset(writeBuffer, true);
 
-                opCtx = rtree.createOpContext(IndexOp.SEARCH);
+                indexAccessor = rtree.createAccessor();
             } catch (Exception e) {
                 writer.fail();
                 throw e;
@@ -154,7 +153,7 @@
 
                 searchPred.setSearchKey(searchKey);
                 cursor.reset();
-                rtree.search(cursor, searchPred, opCtx);
+                indexAccessor.search(cursor, searchPred);
                 writeSearchResults();
             }
         } catch (Exception e) {
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
index 0486496..546a4d4 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
@@ -35,7 +35,7 @@
 public abstract class RTreeNSMFrame extends TreeIndexNSMFrame implements
 		IRTreeFrame {
 	protected static final int pageNsnOff = smFlagOff + 1;
-	protected static final int rightPageOff = pageNsnOff + 4;
+	protected static final int rightPageOff = pageNsnOff + 8;
 
 	protected ITreeIndexTupleReference[] tuples;
 	protected ITreeIndexTupleReference cmpFrameTuple;
@@ -86,7 +86,7 @@
 	@Override
 	public void initBuffer(byte level) {
 		super.initBuffer(level);
-		buf.putInt(pageNsnOff, 0);
+		buf.putLong(pageNsnOff, 0);
 		buf.putInt(rightPageOff, -1);
 	}
 
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 cb553f7..5f78dcc 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
@@ -26,12 +26,15 @@
 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.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.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.IndexType;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
 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.impls.TreeDiskOrderScanCursor;
@@ -84,7 +87,7 @@
         this.cmp = cmp;
         this.freePageManager = freePageManager;
         this.interiorFrameFactory = interiorFrameFactory;
-        this.leafFrameFactory = leafFrameFactory;        
+        this.leafFrameFactory = leafFrameFactory;
         globalNsn = new AtomicLong();
         this.treeLatch = new ReentrantReadWriteLock(true);
         this.diskOrderScanPredicate = new SearchPredicate(null, cmp);
@@ -242,16 +245,13 @@
         fileId = -1;
     }
 
-	@Override
-	public RTreeOpContext createOpContext(IndexOp op) {
-		return new RTreeOpContext(op,
-				(IRTreeLeafFrame) leafFrameFactory.createFrame(),
-				(IRTreeInteriorFrame) interiorFrameFactory.createFrame(),
-				freePageManager.getMetaDataFrameFactory().createFrame(), 8);
-	}
+    private RTreeOpContext createOpContext() {
+        return new RTreeOpContext((IRTreeLeafFrame) leafFrameFactory.createFrame(),
+                (IRTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
+                        .createFrame(), 8);
+    }
 
-    @Override
-    public void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+    private void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException, PageAllocationException {
         RTreeOpContext ctx = (RTreeOpContext) ictx;
         ctx.reset();
         ctx.setTuple(tuple);
@@ -292,115 +292,142 @@
     public ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException {
         int pageId = rootPage;
         boolean writeLatched = false;
+        boolean readLatched = false;
+        boolean succeed = false;
         ICachedPage node = null;
         boolean isLeaf = false;
         long pageLsn = 0, parentLsn = 0;
 
-        while (true) {
-            if (!writeLatched) {
-                node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-                incrementPins();
-                ctx.interiorFrame.setPage(node);
-                isLeaf = ctx.interiorFrame.isLeaf();
-                if (isLeaf) {
-                    node.acquireWriteLatch();
-                    incrementWriteLatchesAcquired();
-                    writeLatched = true;
+        try {
 
-                    if (!ctx.interiorFrame.isLeaf()) {
+            while (true) {
+                if (!writeLatched) {
+                    node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                    incrementPins();
+                    ctx.interiorFrame.setPage(node);
+                    isLeaf = ctx.interiorFrame.isLeaf();
+                    if (isLeaf) {
+                        node.acquireWriteLatch();
+                        writeLatched = true;
+                        incrementWriteLatchesAcquired();
+
+                        if (!ctx.interiorFrame.isLeaf()) {
+                            node.releaseWriteLatch();
+                            writeLatched = false;
+                            incrementWriteLatchesReleased();
+                            bufferCache.unpin(node);
+                            incrementUnpins();
+                            continue;
+                        }
+                    } else {
+                        // Be optimistic and grab read latch first. We will swap
+                        // it
+                        // to write latch if we need to enlarge the best child
+                        // tuple.
+                        node.acquireReadLatch();
+                        readLatched = true;
+                        incrementReadLatchesAcquired();
+                    }
+                }
+
+                if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
+                    // Concurrent split detected, go back to parent and
+                    // re-choose
+                    // the best child
+                    if (writeLatched) {
                         node.releaseWriteLatch();
+                        writeLatched = false;
                         incrementWriteLatchesReleased();
                         bufferCache.unpin(node);
                         incrementUnpins();
-                        writeLatched = false;
-                        continue;
+                    } else {
+                        node.releaseReadLatch();
+                        readLatched = false;
+                        incrementReadLatchesReleased();
+                        bufferCache.unpin(node);
+                        incrementUnpins();
                     }
-                } else {
-                    // Be optimistic and grab read latch first. We will swap it
-                    // to write latch if we need to enlarge the best child
-                    // tuple.
-                    node.acquireReadLatch();
-                    incrementReadLatchesAcquired();
-                }
-            }
 
-            if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
-                // Concurrent split detected, go back to parent and re-choose
-                // the best child
-                if (writeLatched) {
+                    pageId = ctx.pathList.getLastPageId();
+                    if (pageId != rootPage) {
+                        parentLsn = ctx.pathList.getPageLsn(ctx.pathList.size() - 2);
+                    }
+                    ctx.pathList.moveLast();
+                    continue;
+                }
+
+                pageLsn = ctx.interiorFrame.getPageLsn();
+                ctx.pathList.add(pageId, pageLsn, -1);
+
+                if (!isLeaf) {
+                    // findBestChild must be called *before* getBestChildPageId
+                    ctx.interiorFrame.findBestChild(ctx.getTuple(), cmp);
+                    int childPageId = ctx.interiorFrame.getBestChildPageId();
+
+                    if (!writeLatched) {
+                        node.releaseReadLatch();
+                        readLatched = false;
+                        incrementReadLatchesReleased();
+                        // TODO: do we need to un-pin and pin again?
+                        bufferCache.unpin(node);
+                        incrementUnpins();
+
+                        node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                        incrementPins();
+                        node.acquireWriteLatch();
+                        writeLatched = true;
+                        incrementWriteLatchesAcquired();
+                        ctx.interiorFrame.setPage(node);
+
+                        if (ctx.interiorFrame.getPageLsn() != pageLsn) {
+                            // The page was changed while we unlocked it; thus,
+                            // retry (re-choose best child)
+
+                            ctx.pathList.moveLast();
+                            continue;
+                        }
+                    }
+
+                    // We don't need to reset the frameTuple because it is
+                    // already pointing to the best child
+                    ctx.interiorFrame.enlarge(ctx.getTuple(), cmp);
+
                     node.releaseWriteLatch();
+                    writeLatched = false;
                     incrementWriteLatchesReleased();
                     bufferCache.unpin(node);
                     incrementUnpins();
-                    writeLatched = false;
+
+                    pageId = childPageId;
+                    parentLsn = pageLsn;
                 } else {
-                    node.releaseReadLatch();
-                    incrementReadLatchesReleased();
-                    bufferCache.unpin(node);
-                    incrementUnpins();
+                    ctx.leafFrame.setPage(node);
+                    succeed = true;
+                    return node;
                 }
-
-                pageId = ctx.pathList.getLastPageId();
-                if (pageId != rootPage) {
-                    parentLsn = ctx.pathList.getPageLsn(ctx.pathList.size() - 2);
-                }
-                ctx.pathList.moveLast();
-                continue;
             }
-
-            pageLsn = ctx.interiorFrame.getPageLsn();
-            ctx.pathList.add(pageId, pageLsn, -1);
-
-            if (!isLeaf) {
-                // findBestChild must be called *before* getBestChildPageId
-                ctx.interiorFrame.findBestChild(ctx.getTuple(), cmp);
-                int childPageId = ctx.interiorFrame.getBestChildPageId();
-
-                if (!writeLatched) {
+        } finally {
+            if (!succeed) {
+                if (readLatched) {
                     node.releaseReadLatch();
+                    readLatched = false;
                     incrementReadLatchesReleased();
-                    // TODO: do we need to un-pin and pin again?
                     bufferCache.unpin(node);
                     incrementUnpins();
-
-                    node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-                    incrementPins();
-                    node.acquireWriteLatch();
-                    incrementWriteLatchesAcquired();
-                    ctx.interiorFrame.setPage(node);
-                    writeLatched = true;
-
-                    if (ctx.interiorFrame.getPageLsn() != pageLsn) {
-                        // The page was changed while we unlocked it; thus,
-                        // retry (re-choose best child)
-
-                        ctx.pathList.moveLast();
-                        continue;
-                    }
+                } else if (writeLatched) {
+                    node.releaseWriteLatch();
+                    writeLatched = false;
+                    incrementWriteLatchesReleased();
+                    bufferCache.unpin(node);
+                    incrementUnpins();
                 }
-
-                // We don't need to reset the frameTuple because it is
-                // already pointing to the best child
-                ctx.interiorFrame.enlarge(ctx.getTuple(), cmp);
-
-                node.releaseWriteLatch();
-                incrementWriteLatchesReleased();
-                bufferCache.unpin(node);
-                incrementUnpins();
-                writeLatched = false;
-
-                pageId = childPageId;
-                parentLsn = pageLsn;
-            } else {
-                ctx.leafFrame.setPage(node);
-                return node;
             }
         }
     }
 
     private void insertTuple(ICachedPage node, int pageId, ITupleReference tuple, RTreeOpContext ctx, boolean isLeaf)
-            throws HyracksDataException, TreeIndexException {
-    	FrameOpSpaceStatus spaceStatus;
+            throws HyracksDataException, TreeIndexException, PageAllocationException {
+        FrameOpSpaceStatus spaceStatus;
         if (!isLeaf) {
             spaceStatus = ctx.interiorFrame.hasSpaceInsert(tuple);
         } else {
@@ -525,54 +552,71 @@
         }
     }
 
-    public void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+    public void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
+        boolean writeLatched = false;
         int parentId = ctx.pathList.getLastPageId();
         ICachedPage parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
         incrementPins();
         parentNode.acquireWriteLatch();
+        writeLatched = true;
         incrementWriteLatchesAcquired();
         ctx.interiorFrame.setPage(parentNode);
         boolean foundParent = true;
 
-        if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
-            foundParent = false;
-            while (true) {
-                if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp) != -1) {
-                    // found the parent
-                    foundParent = true;
-                    break;
+        try {
+
+            if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
+                foundParent = false;
+                while (true) {
+                    if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp) != -1) {
+                        // found the parent
+                        foundParent = true;
+                        break;
+                    }
+                    int rightPage = ctx.interiorFrame.getRightPage();
+                    parentNode.releaseWriteLatch();
+                    writeLatched = false;
+                    incrementWriteLatchesReleased();
+                    bufferCache.unpin(parentNode);
+                    incrementUnpins();
+
+                    if (rightPage == -1) {
+                        break;
+                    }
+
+                    parentId = rightPage;
+                    parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
+                    incrementPins();
+                    parentNode.acquireWriteLatch();
+                    writeLatched = true;
+                    incrementWriteLatchesAcquired();
+                    ctx.interiorFrame.setPage(parentNode);
                 }
-                int rightPage = ctx.interiorFrame.getRightPage();
+            }
+            if (foundParent) {
+                ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), -1, cmp);
+                insertTuple(parentNode, parentId, ctx.splitKey.getRightTuple(), ctx, ctx.interiorFrame.isLeaf());
+                ctx.pathList.moveLast();
+
                 parentNode.releaseWriteLatch();
+                writeLatched = false;
                 incrementWriteLatchesReleased();
                 bufferCache.unpin(parentNode);
                 incrementUnpins();
+                return;
+            }
 
-                if (rightPage == -1) {
-                    break;
-                }
-
-                parentId = rightPage;
-                parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
-                incrementPins();
-                parentNode.acquireWriteLatch();
-                incrementWriteLatchesAcquired();
-                ctx.interiorFrame.setPage(parentNode);
+        } finally {
+            if (writeLatched) {
+                parentNode.releaseWriteLatch();
+                writeLatched = false;
+                incrementWriteLatchesReleased();
+                bufferCache.unpin(parentNode);
+                incrementUnpins();
             }
         }
-        if (foundParent) {
-            ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), -1, cmp);
-            insertTuple(parentNode, parentId, ctx.splitKey.getRightTuple(), ctx, ctx.interiorFrame.isLeaf());
-            ctx.pathList.moveLast();
-
-            parentNode.releaseWriteLatch();
-            incrementWriteLatchesReleased();
-            bufferCache.unpin(parentNode);
-            incrementUnpins();
-            return;
-        }
-
-        // very rare situation when the there is a root split, do an exhaustive
+        // very rare situation when the there is a root split, do an
+        // exhaustive
         // breadth-first traversal looking for the parent tuple
 
         ctx.pathList.clear();
@@ -582,48 +626,57 @@
     }
 
     public void findPath(RTreeOpContext ctx) throws HyracksDataException {
+        boolean readLatched = false;
         int pageId = rootPage;
         int parentIndex = -1;
         long parentLsn = 0;
         long pageLsn;
         int pageIndex;
+        ICachedPage node = null;
         ctx.traverseList.add(pageId, -1, parentIndex);
-        while (!ctx.traverseList.isLast()) {
-            pageId = ctx.traverseList.getFirstPageId();
-            parentIndex = ctx.traverseList.getFirstPageIndex();
+        try {
+            while (!ctx.traverseList.isLast()) {
+                pageId = ctx.traverseList.getFirstPageId();
+                parentIndex = ctx.traverseList.getFirstPageIndex();
 
-            ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-            incrementPins();
-            node.acquireReadLatch();
-            incrementReadLatchesAcquired();
-            ctx.interiorFrame.setPage(node);
-            pageLsn = ctx.interiorFrame.getPageLsn();
-            pageIndex = ctx.traverseList.first();
-            ctx.traverseList.setPageLsn(pageIndex, pageLsn);
+                node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                incrementPins();
+                node.acquireReadLatch();
+                readLatched = true;
+                incrementReadLatchesAcquired();
+                ctx.interiorFrame.setPage(node);
+                pageLsn = ctx.interiorFrame.getPageLsn();
+                pageIndex = ctx.traverseList.first();
+                ctx.traverseList.setPageLsn(pageIndex, pageLsn);
 
-            ctx.traverseList.moveFirst();
+                ctx.traverseList.moveFirst();
 
-            if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
-                int rightPage = ctx.interiorFrame.getRightPage();
-                if (rightPage != -1) {
-                    ctx.traverseList.add(rightPage, -1, parentIndex);
+                if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
+                    int rightPage = ctx.interiorFrame.getRightPage();
+                    if (rightPage != -1) {
+                        ctx.traverseList.add(rightPage, -1, parentIndex);
+                    }
                 }
-            }
-            parentLsn = pageLsn;
+                parentLsn = pageLsn;
 
-            if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), ctx.traverseList, pageIndex, cmp) != -1) {
-                fillPath(ctx, pageIndex);
-
+                if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), ctx.traverseList, pageIndex, cmp) != -1) {
+                    fillPath(ctx, pageIndex);
+                    return;
+                }
                 node.releaseReadLatch();
+                readLatched = false;
                 incrementReadLatchesReleased();
                 bufferCache.unpin(node);
                 incrementUnpins();
-                return;
             }
-            node.releaseReadLatch();
-            incrementReadLatchesReleased();
-            bufferCache.unpin(node);
-            incrementUnpins();
+        } finally {
+            if (readLatched) {
+                node.releaseReadLatch();
+                readLatched = false;
+                incrementReadLatchesReleased();
+                bufferCache.unpin(node);
+                incrementUnpins();
+            }
         }
     }
 
@@ -634,9 +687,7 @@
         }
     }
 
-    @Override
-    public void delete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
-        RTreeOpContext ctx = (RTreeOpContext) ictx;
+    public void delete(ITupleReference tuple, RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
         ctx.reset();
         ctx.setTuple(tuple);
         ctx.splitKey.reset();
@@ -665,70 +716,85 @@
     }
 
     public void updateParentForDelete(RTreeOpContext ctx) throws HyracksDataException {
+        boolean writeLatched = false;
         int parentId = ctx.pathList.getLastPageId();
         ICachedPage parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
         incrementPins();
         parentNode.acquireWriteLatch();
+        writeLatched = true;
         incrementWriteLatchesAcquired();
         ctx.interiorFrame.setPage(parentNode);
         boolean foundParent = true;
         int tupleIndex = -1;
 
-        if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
-            foundParent = false;
-            while (true) {
-                tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
-                if (tupleIndex != -1) {
-                    // found the parent
-                    foundParent = true;
-                    break;
+        try {
+            if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
+                foundParent = false;
+                while (true) {
+                    tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
+                    if (tupleIndex != -1) {
+                        // found the parent
+                        foundParent = true;
+                        break;
+                    }
+                    int rightPage = ctx.interiorFrame.getRightPage();
+                    parentNode.releaseWriteLatch();
+                    writeLatched = false;
+                    incrementWriteLatchesReleased();
+                    bufferCache.unpin(parentNode);
+                    incrementUnpins();
+
+                    if (rightPage == -1) {
+                        break;
+                    }
+
+                    parentId = rightPage;
+                    parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
+                    incrementPins();
+                    parentNode.acquireWriteLatch();
+                    writeLatched = true;
+                    incrementWriteLatchesAcquired();
+                    ctx.interiorFrame.setPage(parentNode);
                 }
-                int rightPage = ctx.interiorFrame.getRightPage();
+            }
+            if (foundParent) {
+                if (tupleIndex == -1) {
+                    tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
+                }
+                boolean recomputeMBR = ctx.interiorFrame.recomputeMBR(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
+
+                if (recomputeMBR) {
+                    ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
+                    ctx.pathList.moveLast();
+
+                    incrementGlobalNsn();
+                    ctx.interiorFrame.setPageLsn(getGlobalNsn());
+
+                    ctx.splitKey.reset();
+                    if (!ctx.pathList.isEmpty()) {
+                        ctx.interiorFrame.computeMBR(ctx.splitKey);
+                        ctx.splitKey.setLeftPage(parentId);
+                    }
+                } else {
+                    ctx.pathList.moveLast();
+                    ctx.splitKey.reset();
+                }
+
                 parentNode.releaseWriteLatch();
+                writeLatched = false;
                 incrementWriteLatchesReleased();
                 bufferCache.unpin(parentNode);
                 incrementUnpins();
-
-                if (rightPage == -1) {
-                    break;
-                }
-
-                parentId = rightPage;
-                parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
-                incrementPins();
-                parentNode.acquireWriteLatch();
-                incrementWriteLatchesAcquired();
-                ctx.interiorFrame.setPage(parentNode);
+                return;
             }
-        }
-        if (foundParent) {
-            if (tupleIndex == -1) {
-                tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
+        } finally {
+            if (writeLatched) {
+                parentNode.releaseWriteLatch();
+                writeLatched = false;
+                incrementWriteLatchesReleased();
+                bufferCache.unpin(parentNode);
+                incrementUnpins();
             }
-            boolean recomputeMBR = ctx.interiorFrame.recomputeMBR(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
-
-            if (recomputeMBR) {
-                ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
-                ctx.pathList.moveLast();
-
-                incrementGlobalNsn();
-                ctx.interiorFrame.setPageLsn(getGlobalNsn());
-
-                ctx.splitKey.reset();
-                if (!ctx.pathList.isEmpty()) {
-                    ctx.interiorFrame.computeMBR(ctx.splitKey);
-                    ctx.splitKey.setLeftPage(parentId);
-                }
-            } else {
-                ctx.pathList.moveLast();
-                ctx.splitKey.reset();
-            }
-
-            parentNode.releaseWriteLatch();
-            incrementWriteLatchesReleased();
-            bufferCache.unpin(parentNode);
-            incrementUnpins();
-            return;
         }
 
         // very rare situation when the there is a root split, do an exhaustive
@@ -741,87 +807,116 @@
     }
 
     public int findTupleToDelete(RTreeOpContext ctx) throws HyracksDataException {
-
+        boolean writeLatched = false;
+        boolean readLatched = false;
+        boolean succeed = false;
+        ICachedPage node = null;
         ctx.traverseList.add(rootPage, -1, -1);
         ctx.pathList.add(rootPage, -1, ctx.traverseList.size() - 1);
 
-        while (!ctx.pathList.isEmpty()) {
-            int pageId = ctx.pathList.getLastPageId();
-            long parentLsn = ctx.pathList.getLastPageLsn();
-            int pageIndex = ctx.pathList.getLastPageIndex();
-            ctx.pathList.moveLast();
-            ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-            incrementPins();
-            node.acquireReadLatch();
-            incrementReadLatchesAcquired();
-            ctx.interiorFrame.setPage(node);
-            boolean isLeaf = ctx.interiorFrame.isLeaf();
-            long pageLsn = ctx.interiorFrame.getPageLsn();
-            int parentIndex = ctx.traverseList.getPageIndex(pageIndex);
-            ctx.traverseList.setPageLsn(pageIndex, pageLsn);
+        try {
+            while (!ctx.pathList.isEmpty()) {
+                int pageId = ctx.pathList.getLastPageId();
+                long parentLsn = ctx.pathList.getLastPageLsn();
+                int pageIndex = ctx.pathList.getLastPageIndex();
+                ctx.pathList.moveLast();
+                node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                incrementPins();
+                node.acquireReadLatch();
+                readLatched = true;
+                incrementReadLatchesAcquired();
+                ctx.interiorFrame.setPage(node);
+                boolean isLeaf = ctx.interiorFrame.isLeaf();
+                long pageLsn = ctx.interiorFrame.getPageLsn();
+                int parentIndex = ctx.traverseList.getPageIndex(pageIndex);
+                ctx.traverseList.setPageLsn(pageIndex, pageLsn);
 
-            if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
-                // Concurrent split detected, we need to visit the right page
-                int rightPage = ctx.interiorFrame.getRightPage();
-                if (rightPage != -1) {
-                    ctx.traverseList.add(rightPage, -1, parentIndex);
-                    ctx.pathList.add(rightPage, parentLsn, ctx.traverseList.size() - 1);
-                }
-            }
-
-            if (!isLeaf) {
-                for (int i = 0; i < ctx.interiorFrame.getTupleCount(); i++) {
-                    int childPageId = ctx.interiorFrame.getChildPageIdIfIntersect(ctx.tuple, i, cmp);
-                    if (childPageId != -1) {
-                        ctx.traverseList.add(childPageId, -1, pageIndex);
-                        ctx.pathList.add(childPageId, pageLsn, ctx.traverseList.size() - 1);
+                if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
+                    // Concurrent split detected, we need to visit the right
+                    // page
+                    int rightPage = ctx.interiorFrame.getRightPage();
+                    if (rightPage != -1) {
+                        ctx.traverseList.add(rightPage, -1, parentIndex);
+                        ctx.pathList.add(rightPage, parentLsn, ctx.traverseList.size() - 1);
                     }
                 }
-            } else {
-                ctx.leafFrame.setPage(node);
-                int tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
-                if (tupleIndex != -1) {
 
-                    node.releaseReadLatch();
-                    incrementReadLatchesReleased();
-                    bufferCache.unpin(node);
-                    incrementUnpins();
-
-                    node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-                    incrementPins();
-                    node.acquireWriteLatch();
-                    incrementWriteLatchesAcquired();
+                if (!isLeaf) {
+                    for (int i = 0; i < ctx.interiorFrame.getTupleCount(); i++) {
+                        int childPageId = ctx.interiorFrame.getChildPageIdIfIntersect(ctx.tuple, i, cmp);
+                        if (childPageId != -1) {
+                            ctx.traverseList.add(childPageId, -1, pageIndex);
+                            ctx.pathList.add(childPageId, pageLsn, ctx.traverseList.size() - 1);
+                        }
+                    }
+                } else {
                     ctx.leafFrame.setPage(node);
+                    int tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
+                    if (tupleIndex != -1) {
 
-                    if (ctx.leafFrame.getPageLsn() != pageLsn) {
-                        // The page was changed while we unlocked it
+                        node.releaseReadLatch();
+                        readLatched = false;
+                        incrementReadLatchesReleased();
+                        bufferCache.unpin(node);
+                        incrementUnpins();
 
-                        tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
-                        if (tupleIndex == -1) {
-                            ctx.traverseList.add(pageId, -1, parentIndex);
-                            ctx.pathList.add(pageId, parentLsn, ctx.traverseList.size() - 1);
+                        node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                        incrementPins();
+                        node.acquireWriteLatch();
+                        writeLatched = true;
+                        incrementWriteLatchesAcquired();
+                        ctx.leafFrame.setPage(node);
 
-                            node.releaseWriteLatch();
-                            incrementWriteLatchesReleased();
-                            bufferCache.unpin(node);
-                            incrementUnpins();
-                            continue;
+                        if (ctx.leafFrame.getPageLsn() != pageLsn) {
+                            // The page was changed while we unlocked it
+
+                            tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
+                            if (tupleIndex == -1) {
+                                ctx.traverseList.add(pageId, -1, parentIndex);
+                                ctx.pathList.add(pageId, parentLsn, ctx.traverseList.size() - 1);
+
+                                node.releaseWriteLatch();
+                                writeLatched = false;
+                                incrementWriteLatchesReleased();
+                                bufferCache.unpin(node);
+                                incrementUnpins();
+                                continue;
+                            } else {
+                                ctx.pathList.clear();
+                                fillPath(ctx, pageIndex);
+                                succeed = true;
+                                return tupleIndex;
+                            }
                         } else {
                             ctx.pathList.clear();
                             fillPath(ctx, pageIndex);
+                            succeed = true;
                             return tupleIndex;
                         }
-                    } else {
-                        ctx.pathList.clear();
-                        fillPath(ctx, pageIndex);
-                        return tupleIndex;
                     }
                 }
+                node.releaseReadLatch();
+                readLatched = false;
+                incrementReadLatchesReleased();
+                bufferCache.unpin(node);
+                incrementUnpins();
             }
-            node.releaseReadLatch();
-            incrementReadLatchesReleased();
-            bufferCache.unpin(node);
-            incrementUnpins();
+        } finally {
+            if (!succeed) {
+                if (readLatched) {
+                    node.releaseReadLatch();
+                    readLatched = false;
+                    incrementReadLatchesReleased();
+                    bufferCache.unpin(node);
+                    incrementUnpins();
+                } else if (writeLatched) {
+                    node.releaseWriteLatch();
+                    writeLatched = false;
+                    incrementWriteLatchesReleased();
+                    bufferCache.unpin(node);
+                    incrementUnpins();
+                }
+            }
         }
         return -1;
     }
@@ -838,14 +933,14 @@
         }
     }
 
-    public void search(ITreeIndexCursor cursor, SearchPredicate pred, RTreeOpContext ctx) throws Exception {
-        ctx.reset();
+    private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, RTreeOpContext ctx)  throws HyracksDataException, TreeIndexException {
+    	ctx.reset();
         ctx.cursor = cursor;
 
         cursor.setBufferCache(bufferCache);
         cursor.setFileId(fileId);
         ctx.cursorInitialState.setRootPage(rootPage);
-        ctx.cursor.open(ctx.cursorInitialState, pred);
+        ctx.cursor.open(ctx.cursorInitialState, (SearchPredicate)searchPred);
     }
 
     public ITreeIndexFrameFactory getInteriorFrameFactory() {
@@ -864,18 +959,17 @@
         return freePageManager;
     }
 
-    @Override
-    public void update(ITupleReference tuple, IIndexOpContext ictx) {
+    private void update(ITupleReference tuple, RTreeOpContext ctx) {
         throw new UnsupportedOperationException("RTree Update not implemented.");
     }
 
     public final class BulkLoadContext implements IIndexBulkLoadContext {
 
-        public RTreeOpContext insertOpCtx;
+        public ITreeIndexAccessor indexAccessor;
 
         public BulkLoadContext(float fillFactor, IRTreeFrame leafFrame, IRTreeFrame interiorFrame,
                 ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
-            insertOpCtx = createOpContext(IndexOp.INSERT);
+        	indexAccessor = createAccessor();
         }
     }
 
@@ -885,15 +979,16 @@
             throw new HyracksDataException("Trying to bulk-load RTree but RTree has already been loaded.");
         }
 
-        BulkLoadContext ctx = new BulkLoadContext(fillFactor, (IRTreeFrame) leafFrameFactory.createFrame(), (IRTreeFrame) interiorFrameFactory.createFrame(),
-                freePageManager.getMetaDataFrameFactory().createFrame());
+        BulkLoadContext ctx = new BulkLoadContext(fillFactor, (IRTreeFrame) leafFrameFactory.createFrame(),
+                (IRTreeFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
+                        .createFrame());
         return ctx;
     }
 
     @Override
     public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
         try {
-            insert(tuple, ((BulkLoadContext) ictx).insertOpCtx);
+        	((BulkLoadContext) ictx).indexAccessor.insert(tuple);
         } catch (Exception e) {
             throw new HyracksDataException("BulkLoad Error");
         }
@@ -904,10 +999,8 @@
         loaded = true;
     }
 
-    @Override
-    public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException {
+    private void diskOrderScan(ITreeIndexCursor icursor, RTreeOpContext ctx) throws HyracksDataException {
         TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
-        RTreeOpContext ctx = (RTreeOpContext) ictx;
         ctx.reset();
 
         int currentPageId = rootPage + 1;
@@ -915,12 +1008,18 @@
 
         ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
         page.acquireReadLatch();
-        cursor.setBufferCache(bufferCache);
-        cursor.setFileId(fileId);
-        cursor.setCurrentPageId(currentPageId);
-        cursor.setMaxPageId(maxPageId);
-        ctx.cursorInitialState.setPage(page);
-        cursor.open(ctx.cursorInitialState, diskOrderScanPredicate);
+        try {
+            cursor.setBufferCache(bufferCache);
+            cursor.setFileId(fileId);
+            cursor.setCurrentPageId(currentPageId);
+            cursor.setMaxPageId(maxPageId);
+            ctx.cursorInitialState.setPage(page);
+            cursor.open(ctx.cursorInitialState, diskOrderScanPredicate);
+        } catch (Exception e) {
+            page.releaseReadLatch();
+            bufferCache.unpin(page);
+            throw new HyracksDataException(e);
+        }
     }
 
     @Override
@@ -937,4 +1036,50 @@
     public IndexType getIndexType() {
         return IndexType.RTREE;
     }
-}
+    
+    @Override
+	public ITreeIndexAccessor createAccessor() {
+    	return new RTreeAccessor(this);
+	}
+    
+    private class RTreeAccessor implements ITreeIndexAccessor {
+        private RTree rtree;
+        private RTreeOpContext ctx;
+        
+        public RTreeAccessor(RTree rtree) {
+            this.rtree = rtree;
+            this.ctx = rtree.createOpContext();
+        }
+        
+        @Override
+        public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+            ctx.reset(IndexOp.INSERT);
+            rtree.insert(tuple, ctx);
+        }
+
+        @Override
+        public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+            ctx.reset(IndexOp.UPDATE);
+            rtree.update(tuple, ctx);
+        }
+
+        @Override
+        public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+            ctx.reset(IndexOp.DELETE);
+            rtree.delete(tuple, ctx);
+        }
+
+        @Override
+        public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+                TreeIndexException {
+            ctx.reset(IndexOp.SEARCH);
+            rtree.search(cursor, searchPred, ctx);
+        }
+
+        @Override
+        public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
+            ctx.reset(IndexOp.DISKORDERSCAN);
+            rtree.diskOrderScan(cursor, ctx);
+        }
+    }
+}
\ No newline at end of file
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 dc4a753..c258377 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
@@ -37,24 +37,13 @@
 	public PathList traverseList; // used for traversing the tree
 	private static final int initTraverseListSize = 100;
 
-	public RTreeOpContext(IndexOp op, IRTreeLeafFrame leafFrame,
+	public RTreeOpContext(IRTreeLeafFrame leafFrame,
 			IRTreeInteriorFrame interiorFrame,
 			ITreeIndexMetaDataFrame metaFrame, int treeHeightHint) {
 		this.interiorFrame = interiorFrame;
 		this.leafFrame = leafFrame;
 		this.metaFrame = metaFrame;
 		pathList = new PathList(treeHeightHint, treeHeightHint);
-		if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
-			splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter()
-					.createTupleReference(), interiorFrame.getTupleWriter()
-					.createTupleReference());
-			traverseList = new PathList(initTraverseListSize,
-					initTraverseListSize);
-		} else {
-			splitKey = null;
-			traverseList = null;
-			cursorInitialState = new RTreeCursorInitialState(pathList, 1);
-		}
 	}
 
 	public ITupleReference getTuple() {
@@ -76,6 +65,9 @@
 
 	@Override
 	public void reset(IndexOp newOp) {
+		if (op != null && newOp == op) {
+			return;
+		}
 		if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
 			if (splitKey == null) {
 				splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter()
@@ -86,11 +78,9 @@
 				traverseList = new PathList(initTraverseListSize,
 						initTraverseListSize);
 			}
-
-		} else {
-			if (cursorInitialState == null) {
-				cursorInitialState = new RTreeCursorInitialState(pathList, 1);
-			}
+		}
+		if (cursorInitialState == null) {
+			cursorInitialState = new RTreeCursorInitialState(pathList, 1);
 		}
 		this.op = newOp;
 	}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index a138212..82f07ad 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -30,188 +30,193 @@
 
 public class RTreeSearchCursor implements ITreeIndexCursor {
 
-	private int fileId = -1;
-	private ICachedPage page = null;
-	private IRTreeInteriorFrame interiorFrame = null;
-	private IRTreeLeafFrame leafFrame = null;
-	private IBufferCache bufferCache = null;
+    private int fileId = -1;
+    private ICachedPage page = null;
+    private IRTreeInteriorFrame interiorFrame = null;
+    private IRTreeLeafFrame leafFrame = null;
+    private IBufferCache bufferCache = null;
 
-	private SearchPredicate pred;
-	private PathList pathList;
-	private int rootPage;
-	ITupleReference searchKey;
+    private SearchPredicate pred;
+    private PathList pathList;
+    private int rootPage;
+    ITupleReference searchKey;
 
-	private int tupleIndex = 0;
-	private int tupleIndexInc = 0;
+    private int tupleIndex = 0;
+    private int tupleIndexInc = 0;
 
-	private MultiComparator cmp;
+    private MultiComparator cmp;
 
-	private ITreeIndexTupleReference frameTuple;
-	private boolean readLatched = false;
+    private ITreeIndexTupleReference frameTuple;
+    private boolean readLatched = false;
 
-	private int pin = 0;
-	private int unpin = 0;
+    private int pin = 0;
+    private int unpin = 0;
 
-	public RTreeSearchCursor(IRTreeInteriorFrame interiorFrame,
-			IRTreeLeafFrame leafFrame) {
-		this.interiorFrame = interiorFrame;
-		this.leafFrame = leafFrame;
-		this.frameTuple = leafFrame.createTupleReference();
-	}
+    public RTreeSearchCursor(IRTreeInteriorFrame interiorFrame, IRTreeLeafFrame leafFrame) {
+        this.interiorFrame = interiorFrame;
+        this.leafFrame = leafFrame;
+        this.frameTuple = leafFrame.createTupleReference();
+    }
 
-	@Override
-	public void close() throws Exception {
-		if (readLatched) {
-			page.releaseReadLatch();
-			bufferCache.unpin(page);
-			readLatched = false;
-		}
-		tupleIndex = 0;
-		tupleIndexInc = 0;
-		page = null;
-		pathList = null;
-	}
+    @Override
+    public void close() throws Exception {
+        if (readLatched) {
+            page.releaseReadLatch();
+            bufferCache.unpin(page);
+            readLatched = false;
+        }
+        tupleIndex = 0;
+        tupleIndexInc = 0;
+        page = null;
+        pathList = null;
+    }
 
-	public ITupleReference getTuple() {
-		return frameTuple;
-	}
+    public ITupleReference getTuple() {
+        return frameTuple;
+    }
 
-	@Override
-	public ICachedPage getPage() {
-		return page;
-	}
+    @Override
+    public ICachedPage getPage() {
+        return page;
+    }
 
-	public boolean fetchNextLeafPage() throws HyracksDataException {
-		if (readLatched) {
-			page.releaseReadLatch();
-			bufferCache.unpin(page);
-			unpin++;
-			readLatched = false;
-		}
-		while (!pathList.isEmpty()) {
-			int pageId = pathList.getLastPageId();
-			long parentLsn = pathList.getLastPageLsn();
-			pathList.moveLast();
-			ICachedPage node = bufferCache.pin(
-					BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-			pin++;
-			node.acquireReadLatch();
-			readLatched = true;
-			interiorFrame.setPage(node);
-			boolean isLeaf = interiorFrame.isLeaf();
-			long pageLsn = interiorFrame.getPageLsn();
+    public boolean fetchNextLeafPage() throws HyracksDataException {
+        boolean succeed = false;
+        if (readLatched) {
+            page.releaseReadLatch();
+            bufferCache.unpin(page);
+            unpin++;
+            readLatched = false;
+        }
 
-			if (pageId != rootPage && parentLsn < interiorFrame.getPageNsn()) {
-				// Concurrent split detected, we need to visit the right page
-				int rightPage = interiorFrame.getRightPage();
-				if (rightPage != -1) {
-					pathList.add(rightPage, parentLsn, -1);
-				}
-			}
+        while (!pathList.isEmpty()) {
+            int pageId = pathList.getLastPageId();
+            long parentLsn = pathList.getLastPageLsn();
+            pathList.moveLast();
+            ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+            pin++;
+            node.acquireReadLatch();
+            readLatched = true;
+            try {
+                interiorFrame.setPage(node);
+                boolean isLeaf = interiorFrame.isLeaf();
+                long pageLsn = interiorFrame.getPageLsn();
 
-			if (!isLeaf) {
-				for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
-					int childPageId = interiorFrame.getChildPageIdIfIntersect(
-							searchKey, i, cmp);
-					if (childPageId != -1) {
-						pathList.add(childPageId, pageLsn, -1);
-					}
-				}
-			} else {
-				page = node;
-				leafFrame.setPage(page);
-				tupleIndex = 0;
-				return true;
-			}
-			node.releaseReadLatch();
-			readLatched = false;
-			bufferCache.unpin(node);
-			unpin++;
-		}
-		return false;
-	}
+                if (pageId != rootPage && parentLsn < interiorFrame.getPageNsn()) {
+                    // Concurrent split detected, we need to visit the right
+                    // page
+                    int rightPage = interiorFrame.getRightPage();
+                    if (rightPage != -1) {
+                        pathList.add(rightPage, parentLsn, -1);
+                    }
+                }
 
-	@Override
-	public boolean hasNext() throws Exception {
-		if (page == null) {
-			return false;
-		}
+                if (!isLeaf) {
+                    for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
+                        int childPageId = interiorFrame.getChildPageIdIfIntersect(searchKey, i, cmp);
+                        if (childPageId != -1) {
+                            pathList.add(childPageId, pageLsn, -1);
+                        }
+                    }
+                } else {
+                    page = node;
+                    leafFrame.setPage(page);
+                    tupleIndex = 0;
+                    succeed = true;
+                    return true;
+                }
+            } finally {
+                if (!succeed) {
+                    if (readLatched) {
+                        node.releaseReadLatch();
+                        readLatched = false;
+                        bufferCache.unpin(node);
+                        unpin++;
+                    }
+                }
+            }
+        }
+        return false;
+    }
 
-		if (tupleIndex == leafFrame.getTupleCount()) {
-			if (!fetchNextLeafPage()) {
-				return false;
-			}
-		}
+    @Override
+    public boolean hasNext() throws Exception {
+        if (page == null) {
+            return false;
+        }
 
-		do {
-			for (int i = tupleIndex; i < leafFrame.getTupleCount(); i++) {
-				if (leafFrame.intersect(searchKey, i, cmp)) {
-					frameTuple.resetByTupleIndex(leafFrame, i);
-					tupleIndexInc = i + 1;
-					return true;
-				}
-			}
-		} while (fetchNextLeafPage());
-		return false;
-	}
+        if (tupleIndex == leafFrame.getTupleCount()) {
+            if (!fetchNextLeafPage()) {
+                return false;
+            }
+        }
 
-	@Override
-	public void next() throws Exception {
-		tupleIndex = tupleIndexInc;
-	}
+        do {
+            for (int i = tupleIndex; i < leafFrame.getTupleCount(); i++) {
+                if (leafFrame.intersect(searchKey, i, cmp)) {
+                    frameTuple.resetByTupleIndex(leafFrame, i);
+                    tupleIndexInc = i + 1;
+                    return true;
+                }
+            }
+        } while (fetchNextLeafPage());
+        return false;
+    }
 
-	@Override
-	public void open(ICursorInitialState initialState,
-			ISearchPredicate searchPred) throws Exception {
-		// in case open is called multiple times without closing
-		if (this.page != null) {
-			this.page.releaseReadLatch();
-			readLatched = false;
-			bufferCache.unpin(this.page);
-			pathList.clear();
-		}
+    @Override
+    public void next() throws Exception {
+        tupleIndex = tupleIndexInc;
+    }
 
-		pathList = ((RTreeCursorInitialState) initialState).getPathList();
-		rootPage = ((RTreeCursorInitialState) initialState).getRootPage();
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // in case open is called multiple times without closing
+        if (this.page != null) {
+            this.page.releaseReadLatch();
+            readLatched = false;
+            bufferCache.unpin(this.page);
+            pathList.clear();
+        }
 
-		pred = (SearchPredicate) searchPred;
-		cmp = pred.getLowKeyComparator();
-		searchKey = pred.getSearchKey();
+        pathList = ((RTreeCursorInitialState) initialState).getPathList();
+        rootPage = ((RTreeCursorInitialState) initialState).getRootPage();
 
-		int maxFieldPos = cmp.getKeyFieldCount() / 2;
-		for (int i = 0; i < maxFieldPos; i++) {
-			int j = maxFieldPos + i;
-			int c = cmp.getComparators()[i].compare(searchKey.getFieldData(i),
-					searchKey.getFieldStart(i), searchKey.getFieldLength(i),
-					searchKey.getFieldData(j), searchKey.getFieldStart(j),
-					searchKey.getFieldLength(j));
-			if (c > 0) {
-				throw new IllegalArgumentException(
-						"The low key point has larger coordinates than the high key point.");
-			}
-		}
+        pred = (SearchPredicate) searchPred;
+        cmp = pred.getLowKeyComparator();
+        searchKey = pred.getSearchKey();
 
-		pathList.add(this.rootPage, -1, -1);
-		tupleIndex = 0;
-		fetchNextLeafPage();
-	}
+        int maxFieldPos = cmp.getKeyFieldCount() / 2;
+        for (int i = 0; i < maxFieldPos; i++) {
+            int j = maxFieldPos + i;
+            int c = cmp.getComparators()[i].compare(searchKey.getFieldData(i), searchKey.getFieldStart(i),
+                    searchKey.getFieldLength(i), searchKey.getFieldData(j), searchKey.getFieldStart(j),
+                    searchKey.getFieldLength(j));
+            if (c > 0) {
+                throw new IllegalArgumentException("The low key point has larger coordinates than the high key point.");
+            }
+        }
 
-	@Override
-	public void reset() {
-		try {
-			close();
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-	}
+        pathList.add(this.rootPage, -1, -1);
+        tupleIndex = 0;
+        fetchNextLeafPage();
+    }
 
-	@Override
-	public void setBufferCache(IBufferCache bufferCache) {
-		this.bufferCache = bufferCache;
-	}
+    @Override
+    public void reset() {
+        try {
+            close();
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
 
-	@Override
-	public void setFileId(int fileId) {
-		this.fileId = fileId;
-	}
+    @Override
+    public void setBufferCache(IBufferCache bufferCache) {
+        this.bufferCache = bufferCache;
+    }
+
+    @Override
+    public void setFileId(int fileId) {
+        this.fileId = fileId;
+    }
 }
\ No newline at end of file
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index d7558ba..883b25e 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -132,6 +132,9 @@
         pinSanityCheck(dpid);
         
         CachedPage cPage = findPage(dpid, newPage);
+        if (cPage == null) {
+        	throw new HyracksDataException("Failed to pin page because all pages are pinned.");
+        }
         if (!newPage) {
             if (!cPage.valid) {
                 /*
@@ -564,8 +567,11 @@
         synchronized (fileInfoMap) {
             try {
                 for(Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
-                    sweepAndFlush(entry.getKey());
-                    ioManager.close(entry.getValue().getFileHandle());
+                	boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
+                    sweepAndFlush(entry.getKey(), !fileHasBeenDeleted);                            
+                    if (!fileHasBeenDeleted) {
+                    	ioManager.close(entry.getValue().getFileHandle());
+                    }
                 }
             } catch(HyracksDataException e) {
                 e.printStackTrace();
@@ -601,9 +607,12 @@
                     for(Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
                         if(entry.getValue().getReferenceCount() <= 0) {
                             int entryFileId = entry.getKey();
-                            sweepAndFlush(entryFileId);
+                            boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
+                            sweepAndFlush(entryFileId, !fileHasBeenDeleted);                            
+                            if (!fileHasBeenDeleted) {
+                            	ioManager.close(entry.getValue().getFileHandle());
+                            }
                             fileInfoMap.remove(entryFileId);
-                            ioManager.close(entry.getValue().getFileHandle());
                             unreferencedFileFound = true;
                             // for-each iterator is invalid because we changed fileInfoMap
                             break;
@@ -626,7 +635,7 @@
         }
     }
         
-    private void sweepAndFlush(int fileId) throws HyracksDataException {
+    private void sweepAndFlush(int fileId, boolean flushDirtyPages) throws HyracksDataException {
         for (int i = 0; i < pageMap.length; ++i) {
             CacheBucket bucket = pageMap[i];
             bucket.bucketLock.lock();
@@ -637,7 +646,7 @@
                     if (cPage == null) {
                         break;
                     }
-                    if (invalidateIfFileIdMatch(fileId, cPage)) {
+                    if (invalidateIfFileIdMatch(fileId, cPage, flushDirtyPages)) {
                         prev.next = cPage.next;
                         cPage.next = null;
                     } else {
@@ -646,7 +655,7 @@
                 }
                 // Take care of the head of the chain.
                 if (bucket.cachedPage != null) {
-                    if (invalidateIfFileIdMatch(fileId, bucket.cachedPage)) {
+                    if (invalidateIfFileIdMatch(fileId, bucket.cachedPage, flushDirtyPages)) {
                         CachedPage cPage = bucket.cachedPage;
                         bucket.cachedPage = bucket.cachedPage.next;
                         cPage.next = null;
@@ -658,14 +667,19 @@
         }
     }
 
-    private boolean invalidateIfFileIdMatch(int fileId, CachedPage cPage) throws HyracksDataException {
+    private boolean invalidateIfFileIdMatch(int fileId, CachedPage cPage, boolean flushDirtyPages) throws HyracksDataException {
         if (BufferedFileHandle.getFileId(cPage.dpid) == fileId) {
-            if (cPage.dirty.get()) {
-                write(cPage);
+            int pinCount;
+        	if (cPage.dirty.get()) {
+				if (flushDirtyPages) {
+					write(cPage);
+				}
                 cPage.dirty.set(false);
-                cPage.pinCount.decrementAndGet();
+                pinCount = cPage.pinCount.decrementAndGet();
+            } else {
+            	pinCount = cPage.pinCount.get();
             }
-            if (cPage.pinCount.get() != 0) {
+            if (pinCount != 0) {
                 throw new IllegalStateException("Page is pinned and file is being closed");
             }
             cPage.invalidate();
@@ -697,11 +711,22 @@
             LOGGER.info("Deleting file: " + fileId + " in cache: " + this);
         }
         synchronized (fileInfoMap) {
-            BufferedFileHandle fInfo = fileInfoMap.get(fileId);
-            if (fInfo != null) {
-                throw new HyracksDataException("Deleting open file");
-            }
-            fileMapManager.unregisterFile(fileId);
+        	BufferedFileHandle fInfo = null;
+        	try {
+				fInfo = fileInfoMap.get(fileId);
+				if (fInfo != null && fInfo.getReferenceCount() > 0) {
+					throw new HyracksDataException("Deleting open file");
+				}
+			} finally {
+				fileMapManager.unregisterFile(fileId);
+				if (fInfo != null) {
+					// Mark the fInfo as deleted, 
+					// such that when its pages are reclaimed in openFile(),
+					// the pages are not flushed to disk but only invalidates.
+					ioManager.close(fInfo.getFileHandle());
+					fInfo.markAsDeleted();
+				}
+			}       
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
index 3137f20..ac062d2 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
@@ -37,6 +37,14 @@
         return handle;
     }
 
+    public void markAsDeleted() {
+    	handle = null;
+    }
+    
+    public boolean fileHasBeenDeleted() {
+    	return handle == null;
+    }
+    
     public int incReferenceCount() {
         return refCount.incrementAndGet();
     }
diff --git a/hyracks-test-support/pom.xml b/hyracks-test-support/pom.xml
index 95b3937..fdcf7d4 100644
--- a/hyracks-test-support/pom.xml
+++ b/hyracks-test-support/pom.xml
@@ -43,5 +43,12 @@
   		<type>jar</type>
   		<scope>compile</scope>
   	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-storage-am-invertedindex</artifactId>
+  		<version>0.2.0-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
   </dependencies>
 </project>
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestInvertedIndexRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestInvertedIndexRegistryProvider.java
new file mode 100644
index 0000000..c7ee98e
--- /dev/null
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestInvertedIndexRegistryProvider.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.test.support;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+
+public class TestInvertedIndexRegistryProvider implements IIndexRegistryProvider<InvertedIndex> {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IndexRegistry<InvertedIndex> getRegistry(IHyracksTaskContext ctx) {
+        return TestStorageManagerComponentHolder.getInvertedIndexRegistry(ctx);
+    }
+}
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 c7bd09d..a0aa44b 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
@@ -17,6 +17,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 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.am.invertedindex.impls.InvertedIndex;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -31,6 +32,7 @@
     private static IBufferCache bufferCache;
     private static IFileMapProvider fileMapProvider;
     private static IndexRegistry<ITreeIndex> treeIndexRegistry;
+    private static IndexRegistry<InvertedIndex> invIndexRegistry;
 
     private static int pageSize;
     private static int numPages;
@@ -43,6 +45,7 @@
         bufferCache = null;
         fileMapProvider = null;
         treeIndexRegistry = null;
+        invIndexRegistry = null;
     }
 
     public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
@@ -69,4 +72,11 @@
         }
         return treeIndexRegistry;
     }
+    
+    public synchronized static IndexRegistry<InvertedIndex> getInvertedIndexRegistry(IHyracksTaskContext ctx) {
+        if (invIndexRegistry == null) {
+        	invIndexRegistry = new IndexRegistry<InvertedIndex>();
+        }
+        return invIndexRegistry;
+    }
 }
\ 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/BTreeFieldPrefixNSMTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
index 0f3edd6..5f2203e 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
@@ -18,6 +18,7 @@
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
 import java.util.Random;
+import java.util.logging.Level;
 
 import org.junit.Assert;
 import org.junit.Test;
@@ -56,8 +57,11 @@
 
     private ITupleReference createTuple(IHyracksTaskContext ctx, int f0, int f1, int f2, boolean print)
             throws HyracksDataException {
-        if (print)
-            LOGGER.info("CREATING: " + f0 + " " + f1 + " " + f2);
+        if (print) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("CREATING: " + f0 + " " + f1 + " " + f2);
+            }
+        }
 
         ByteBuffer buf = ctx.allocateFrame();
         FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -135,8 +139,11 @@
             // insert records with random calls to compact and compress
             for (int i = 0; i < numRecords; i++) {
 
-                if ((i + 1) % 100 == 0)
-                    LOGGER.info("INSERTING " + (i + 1) + " / " + numRecords);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    if ((i + 1) % 100 == 0) {
+                        LOGGER.info("INSERTING " + (i + 1) + " / " + numRecords);
+                    }
+                }
 
                 int a = rnd.nextInt() % smallMax;
                 int b = rnd.nextInt() % smallMax;
@@ -174,9 +181,11 @@
 
             // delete records with random calls to compact and compress
             for (int i = 0; i < numRecords; i++) {
-
-                if ((i + 1) % 100 == 0)
-                    LOGGER.info("DELETING " + (i + 1) + " / " + numRecords);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    if ((i + 1) % 100 == 0) {
+                        LOGGER.info("DELETING " + (i + 1) + " / " + numRecords);
+                    }
+                }
 
                 ITupleReference tuple = createTuple(ctx, savedFields[i][0], savedFields[i][1], savedFields[i][2], false);
                 try {
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 7c181c8..f9749cf 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
@@ -4,6 +4,7 @@
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.Random;
+import java.util.logging.Level;
 
 import org.junit.Test;
 
@@ -26,19 +27,18 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 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.util.AbstractBTreeTest;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 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.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexBufferCacheWarmup;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
@@ -80,7 +80,7 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -102,7 +102,9 @@
 
         long start = System.currentTimeMillis();
 
-        LOGGER.info("INSERTING INTO TREE");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("INSERTING INTO TREE");
+        }
 
         ByteBuffer frame = ctx.allocateFrame();
         FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -116,8 +118,7 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
         // 10000
         for (int i = 0; i < 100000; i++) {
 
@@ -135,13 +136,15 @@
 
             tuple.reset(accessor, 0);
 
-            if (i % 10000 == 0) {
-                long end = System.currentTimeMillis();
-                LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 10000 == 0) {
+                    long end = System.currentTimeMillis();
+                    LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+                }
             }
 
             try {
-                btree.insert(tuple, insertOpCtx);
+                indexAccessor.insert(tuple);
             } catch (TreeIndexException e) {
             } catch (Exception e) {
                 e.printStackTrace();
@@ -151,7 +154,9 @@
         TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
                 btree.getRootPageId());
         TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
-        LOGGER.info("\n" + stats.toString());
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("\n" + stats.toString());
+        }
 
         TreeIndexBufferCacheWarmup bufferCacheWarmup = new TreeIndexBufferCacheWarmup(bufferCache, freePageManager,
                 fileId);
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 a357f2a..e5ed5c2 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
@@ -23,6 +23,7 @@
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
+import java.util.logging.Level;
 
 import org.junit.Test;
 
@@ -52,13 +53,12 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 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.util.AbstractBTreeTest;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
 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.ITreeIndexAccessor;
 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;
@@ -68,9 +68,9 @@
 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.impls.TreeDiskOrderScanCursor;
-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.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
 import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
@@ -92,7 +92,9 @@
     @Test
     public void test01() throws Exception {
 
-        LOGGER.info("FIXED-LENGTH KEY TEST");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("FIXED-LENGTH KEY TEST");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -113,7 +115,7 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -135,7 +137,9 @@
 
         long start = System.currentTimeMillis();
 
-        LOGGER.info("INSERTING INTO TREE");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("INSERTING INTO TREE");
+        }
 
         ByteBuffer frame = ctx.allocateFrame();
         FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -149,8 +153,8 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        
         // 10000
         for (int i = 0; i < 10000; i++) {
 
@@ -171,13 +175,15 @@
             ArrayTupleReference t = new ArrayTupleReference();
             t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
             
-            if (i % 1000 == 0) {
-                long end = System.currentTimeMillis();
-                LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    long end = System.currentTimeMillis();
+                    LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+                }
             }
 
             try {
-                btree.insert(t, insertOpCtx);
+                indexAccessor.insert(t);
             } catch (TreeIndexException e) {
             } catch (Exception e) {
                 e.printStackTrace();
@@ -186,25 +192,31 @@
         // btree.printTree(leafFrame, interiorFrame);
 
         int maxPage = btree.getFreePageManager().getMaxPage(metaFrame);
-        LOGGER.info("MAXPAGE: " + maxPage);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("MAXPAGE: " + maxPage);
+        }
 
         long end = System.currentTimeMillis();
         long duration = end - start;
-        LOGGER.info("DURATION: " + duration);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DURATION: " + duration);
+        }
 
         // ordered scan
-
-        LOGGER.info("ORDERED SCAN:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("ORDERED SCAN:");
+        }
         ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(scanCursor, nullPred, searchOpCtx);
+        indexAccessor.search(scanCursor, nullPred);
         try {
             while (scanCursor.hasNext()) {
                 scanCursor.next();
                 ITupleReference frameTuple = scanCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -213,16 +225,19 @@
         }
 
         // disk-order scan
-        LOGGER.info("DISK-ORDER SCAN:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DISK-ORDER SCAN:");
+        }
         TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
-        BTreeOpContext diskOrderScanOpCtx = btree.createOpContext(IndexOp.DISKORDERSCAN);
-        btree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+        indexAccessor.diskOrderScan(diskOrderCursor);
         try {
             while (diskOrderCursor.hasNext()) {
                 diskOrderCursor.next();
                 ITupleReference frameTuple = diskOrderCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -231,7 +246,9 @@
         }
 
         // range search in [-1000, 1000]
-        LOGGER.info("RANGE SEARCH:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RANGE SEARCH:");
+        }
 
         ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
@@ -270,14 +287,16 @@
         MultiComparator searchCmp = new MultiComparator(searchCmps);
 
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
-        btree.search(rangeCursor, rangePred, searchOpCtx);
+        indexAccessor.search(rangeCursor, rangePred);
 
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
                 ITupleReference frameTuple = rangeCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -298,7 +317,9 @@
     @Test
     public void test02() throws Exception {
 
-        LOGGER.info("COMPOSITE KEY TEST");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("COMPOSITE KEY TEST");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -321,7 +342,7 @@
         cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
         cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -343,7 +364,9 @@
 
         long start = System.currentTimeMillis();
 
-        LOGGER.info("INSERTING INTO TREE");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("INSERTING INTO TREE");
+        }
 
         ByteBuffer frame = ctx.allocateFrame();
         FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -357,7 +380,7 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
         
         // Magic test number: 3029. 6398. 4875.
         for (int i = 0; i < 10000; i++) {
@@ -378,12 +401,14 @@
 
             tuple.reset(accessor, 0);
 
-            if (i % 1000 == 0) {
-                LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1);
+                }
             }
             
             try {
-                btree.insert(tuple, insertOpCtx);
+                indexAccessor.insert(tuple);
             } catch (Exception e) {
             }
             
@@ -394,21 +419,25 @@
 
         long end = System.currentTimeMillis();
         long duration = end - start;
-        LOGGER.info("DURATION: " + duration);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DURATION: " + duration);
+        }
 
         // try a simple index scan
-        LOGGER.info("ORDERED SCAN:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("ORDERED SCAN:");
+        }
         ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(scanCursor, nullPred, searchOpCtx);
-
+        indexAccessor.search(scanCursor, nullPred);
         try {
             while (scanCursor.hasNext()) {
                 scanCursor.next();
                 ITupleReference frameTuple = scanCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -417,7 +446,9 @@
         }
 
         // range search in [(-3),(3)]
-        LOGGER.info("RANGE SEARCH:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RANGE SEARCH:");
+        }
         ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
         // build low and high keys
@@ -461,14 +492,16 @@
         // searching
 
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
-        btree.search(rangeCursor, rangePred, searchOpCtx);
+        indexAccessor.search(rangeCursor, rangePred);
 
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
                 ITupleReference frameTuple = rangeCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -476,7 +509,6 @@
             rangeCursor.close();
         }
 
-
         btree.close();
         bufferCache.closeFile(fileId);
         bufferCache.close();
@@ -490,7 +522,9 @@
     @Test
     public void test03() throws Exception {
 
-        LOGGER.info("VARIABLE-LENGTH KEY TEST");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("VARIABLE-LENGTH KEY TEST");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -511,7 +545,7 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -543,7 +577,7 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
         int maxLength = 10; // max string length to be generated
         for (int i = 0; i < 10000; i++) {
 
@@ -561,32 +595,38 @@
 
             tuple.reset(accessor, 0);
 
-            if (i % 1000 == 0) {
-                LOGGER.info("INSERTING " + i);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i);
+                }
             }
 
             try {
-                btree.insert(tuple, insertOpCtx);
+                indexAccessor.insert(tuple);
             } catch (Exception e) {
             }
         }
         // btree.printTree();
 
-        LOGGER.info("DONE INSERTING");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DONE INSERTING");
+        }
 
         // ordered scan
-        LOGGER.info("ORDERED SCAN:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("ORDERED SCAN:");
+        }
         ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(scanCursor, nullPred, searchOpCtx);
-
+        indexAccessor.search(scanCursor, nullPred);
         try {
             while (scanCursor.hasNext()) {
                 scanCursor.next();
                 ITupleReference frameTuple = scanCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -595,7 +635,9 @@
         }
 
         // range search in ["cbf", cc7"]
-        LOGGER.info("RANGE SEARCH:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RANGE SEARCH:");
+        }
 
         ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
@@ -634,14 +676,16 @@
         MultiComparator searchCmp = new MultiComparator(searchCmps);
 
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
-        btree.search(rangeCursor, rangePred, searchOpCtx);
+        indexAccessor.search(rangeCursor, rangePred);
 
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
                 ITupleReference frameTuple = rangeCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -663,7 +707,9 @@
     @Test
     public void test04() throws Exception {
 
-        LOGGER.info("DELETION TEST");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DELETION TEST");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -684,7 +730,7 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -716,15 +762,15 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-        BTreeOpContext deleteOpCtx = btree.createOpContext(IndexOp.DELETE);
-
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        
         int runs = 3;
         for (int run = 0; run < runs; run++) {
 
-            LOGGER.info("DELETION TEST RUN: " + (run + 1) + "/" + runs);
-
-            LOGGER.info("INSERTING INTO BTREE");
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("DELETION TEST RUN: " + (run + 1) + "/" + runs);
+                LOGGER.info("INSERTING INTO BTREE");
+            }
             int maxLength = 10;
             //int ins = 16;
             int ins = 10000;
@@ -750,12 +796,14 @@
 
                 tuple.reset(accessor, 0);
 
-                if (i % 1000 == 0) {
-                    LOGGER.info("INSERTING " + i);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    if (i % 1000 == 0) {
+                        LOGGER.info("INSERTING " + i);
+                    }
                 }
 
                 try {
-                    btree.insert(tuple, insertOpCtx);
+                    indexAccessor.insert(tuple);
                     insDone++;
                 } catch (TreeIndexException e) {
                     //e.printStackTrace();
@@ -766,7 +814,9 @@
                 insDoneCmp[i] = insDone;
             }
             
-            LOGGER.info("DELETING FROM BTREE");
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("DELETING FROM BTREE");
+            }
             int delDone = 0;
             for (int i = 0; i < ins; i++) {
 
@@ -781,12 +831,14 @@
 
                 tuple.reset(accessor, 0);
 
-                if (i % 1000 == 0) {
-                    LOGGER.info("DELETING " + i);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    if (i % 1000 == 0) {
+                        LOGGER.info("DELETING " + i);
+                    }
                 }
 
                 try {
-                    btree.delete(tuple, deleteOpCtx);
+                    indexAccessor.delete(tuple);
                     delDone++;
                 } catch (TreeIndexException e) {
                     //e.printStackTrace();
@@ -795,15 +847,19 @@
                 }
 
                 if (insDoneCmp[i] != delDone) {
-                    LOGGER.info("INCONSISTENT STATE, ERROR IN DELETION TEST");
-                    LOGGER.info("INSDONECMP: " + insDoneCmp[i] + " " + delDone);
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("INCONSISTENT STATE, ERROR IN DELETION TEST");
+                        LOGGER.info("INSDONECMP: " + insDoneCmp[i] + " " + delDone);
+                    }
                     break;
                 }
             }
             // btree.printTree(leafFrame, interiorFrame);
 
             if (insDone != delDone) {
-                LOGGER.info("ERROR! INSDONE: " + insDone + " DELDONE: " + delDone);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("ERROR! INSDONE: " + insDone + " DELDONE: " + delDone);
+                }
                 break;
             }
         }
@@ -816,11 +872,13 @@
     
     private void orderedScan(BTree btree, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, ISerializerDeserializer[] recDescSers) throws Exception {
         // try a simple index scan
-        LOGGER.info("ORDERED SCAN:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("ORDERED SCAN:");
+        }
         ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(scanCursor, nullPred, searchOpCtx);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        indexAccessor.search(scanCursor, nullPred);
         StringBuilder scanResults = new StringBuilder();
         try {
             while (scanCursor.hasNext()) {
@@ -834,7 +892,9 @@
         } finally {
             scanCursor.close();
         }
-        LOGGER.info(scanResults.toString());
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(scanResults.toString());
+        }
     }
     
     // Assuming exactly two BTree fields.
@@ -865,7 +925,9 @@
     @Test
     public void test05() throws Exception {
 
-        LOGGER.info("DELETION TEST");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DELETION TEST");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -886,7 +948,7 @@
         IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
         cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -918,12 +980,13 @@
         accessor.reset(frame);
         FrameTupleReference tuple = new FrameTupleReference();
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-        BTreeOpContext updateOpCtx = btree.createOpContext(IndexOp.UPDATE);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
         Map<String, String> expectedValues = new HashMap<String, String>();
         
-        LOGGER.info("INSERTING INTO BTREE");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("INSERTING INTO BTREE");
+        }
         int maxLength = 10;
         int ins = 10000;
         // Only remember the keys.
@@ -948,11 +1011,13 @@
 
             tuple.reset(accessor, 0);
 
-            if (i % 1000 == 0) {
-                LOGGER.info("INSERTING " + i);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (i % 1000 == 0) {
+                    LOGGER.info("INSERTING " + i);
+                }
             }
             try {
-                btree.insert(t, insertOpCtx);
+                indexAccessor.insert(t);
                 expectedValues.put(f0, f1);
             } catch (TreeIndexException e) {
                 // e.printStackTrace();
@@ -962,8 +1027,7 @@
         }
         ITreeIndexCursor insertCheckCursor = new BTreeRangeSearchCursor(leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(insertCheckCursor, nullPred, searchOpCtx);
+        indexAccessor.search(insertCheckCursor, nullPred);
         try {
             compareActualAndExpected(insertCheckCursor, expectedValues, fieldSerdes);
         } finally {
@@ -972,10 +1036,10 @@
         
         int runs = 3;
         for (int run = 0; run < runs; run++) {
-
-            LOGGER.info("UPDATE TEST RUN: " + (run + 1) + "/" + runs);
-
-            LOGGER.info("UPDATING BTREE");
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("UPDATE TEST RUN: " + (run + 1) + "/" + runs);
+                LOGGER.info("UPDATING BTREE");
+            }
             for (int i = 0; i < ins; i++) {
                 // Generate a new random value for f1.
                 String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
@@ -991,15 +1055,17 @@
 
                 tuple.reset(accessor, 0);
 
-                if (i % 1000 == 0) {
-                    LOGGER.info("UPDATING " + i);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    if (i % 1000 == 0) {
+                        LOGGER.info("UPDATING " + i);
+                    }
                 }
 
                 ArrayTupleReference t = new ArrayTupleReference();
                 t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
                 
                 try {
-                    btree.update(t, updateOpCtx);
+                    indexAccessor.update(t);
                     expectedValues.put(f0s[i], f1);
                 } catch (TreeIndexException e) {
                     e.printStackTrace();
@@ -1009,7 +1075,7 @@
             }
             
             ITreeIndexCursor updateCheckCursor = new BTreeRangeSearchCursor(leafFrame);
-            btree.search(updateCheckCursor, nullPred, searchOpCtx);
+            indexAccessor.search(updateCheckCursor, nullPred);
             try {
                 compareActualAndExpected(updateCheckCursor, expectedValues, fieldSerdes);
             } finally {
@@ -1029,7 +1095,9 @@
     @Test
     public void test06() throws Exception {
 
-        LOGGER.info("BULK LOAD TEST");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BULK LOAD TEST");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -1052,7 +1120,7 @@
         cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
         cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
 
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -1088,7 +1156,9 @@
 
         // generate sorted records
         int ins = 100000;
-        LOGGER.info("BULK LOADING " + ins + " RECORDS");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BULK LOADING " + ins + " RECORDS");
+        }
         long start = System.currentTimeMillis();
         for (int i = 0; i < ins; i++) {
 
@@ -1114,10 +1184,14 @@
 
         long end = System.currentTimeMillis();
         long duration = end - start;
-        LOGGER.info("DURATION: " + duration);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DURATION: " + duration);
+        }
 
         // range search
-        LOGGER.info("RANGE SEARCH:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RANGE SEARCH:");
+        }
         ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
 
         // build low and high keys
@@ -1156,15 +1230,16 @@
 
         // TODO: check when searching backwards
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(rangeCursor, rangePred, searchOpCtx);
-
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
+        indexAccessor.search(rangeCursor, rangePred);
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
                 ITupleReference frameTuple = rangeCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -1183,7 +1258,9 @@
     @Test
     public void test07() throws Exception {
 
-        LOGGER.info("TIME-INTERVAL INTERSECTION DEMO");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TIME-INTERVAL INTERSECTION DEMO");
+        }
 
         TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
         IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
@@ -1206,7 +1283,7 @@
         cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
         cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
         
-        MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+        MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);        
@@ -1273,9 +1350,7 @@
         intervals[9][0] = 20;
         intervals[9][1] = 35;
 
-        BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
-        // int exceptionCount = 0;
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
         for (int i = 0; i < intervalCount; i++) {
             int f0 = intervals[i][0];
             int f1 = intervals[i][1];
@@ -1294,10 +1369,12 @@
 
             tuple.reset(accessor, 0);
 
-            LOGGER.info("INSERTING " + i);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("INSERTING " + i);
+            }
 
             try {
-                btree.insert(tuple, insertOpCtx);
+                indexAccessor.insert(tuple);
             } catch (Exception e) {
             }
         }
@@ -1306,16 +1383,17 @@
 
         long end = System.currentTimeMillis();
         long duration = end - start;
-        LOGGER.info("DURATION: " + duration);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("DURATION: " + duration);
+        }
 
         // try a simple index scan
-
-        LOGGER.info("ORDERED SCAN:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("ORDERED SCAN:");
+        }
         ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-        btree.search(scanCursor, nullPred, searchOpCtx);
-
+        indexAccessor.search(scanCursor, nullPred);
         try {
             while (scanCursor.hasNext()) {
                 scanCursor.next();
@@ -1331,7 +1409,9 @@
         }
 
         // try a range search
-        LOGGER.info("RANGE SEARCH:");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("RANGE SEARCH:");
+        }
         ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
 
         // build low and high keys
@@ -1375,14 +1455,16 @@
         MultiComparator searchCmp = new MultiComparator(searchCmps);
 
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
-        btree.search(rangeCursor, rangePred, searchOpCtx);
+        indexAccessor.search(rangeCursor, rangePred);
 
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
                 ITupleReference frameTuple = rangeCursor.getTuple();
                 String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-                LOGGER.info(rec);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(rec);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java
index ef1e5e3..1daa273 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java
@@ -1,5 +1,7 @@
 package edu.uci.ics.hyracks.storage.am.btree;
 
+import java.util.logging.Level;
+
 import org.junit.Test;
 
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -20,7 +22,9 @@
     
     @Test
     public void oneIntKeyAndValue() throws Exception {        
-        LOGGER.info("BTree " + getTestOpName() + " Test With One Int Key And Value.");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree " + getTestOpName() + " Test With One Int Key And Value.");
+        }
                 
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         // Range search in [-1000, 1000]
@@ -32,8 +36,10 @@
     }
     
     @Test
-    public void twoIntKeys() throws Exception {        
-        LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys.");
+    public void twoIntKeys() throws Exception {    
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys.");
+        }
         
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         
@@ -50,8 +56,10 @@
     }
     
     @Test
-    public void twoIntKeysAndValues() throws Exception {        
-        LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys And Values.");
+    public void twoIntKeysAndValues() throws Exception {  
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys And Values.");
+        }
         
         ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         
@@ -69,7 +77,9 @@
     
     @Test
     public void oneStringKeyAndValue() throws Exception {        
-        LOGGER.info("BTree " + getTestOpName() + " Test With One String Key And Value.");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree " + getTestOpName() + " Test With One String Key And Value.");
+        }
         
         ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
         
@@ -82,8 +92,10 @@
     }
     
     @Test
-    public void twoStringKeys() throws Exception {        
-        LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys.");
+    public void twoStringKeys() throws Exception {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys.");
+        }
         
         ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
         
@@ -100,8 +112,10 @@
     }
     
     @Test
-    public void twoStringKeysAndValues() throws Exception {        
-        LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys And Values.");
+    public void twoStringKeysAndValues() throws Exception {      
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys And Values.");
+        }
         
         ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
         
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 14246f7..79134ae 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
@@ -22,6 +22,7 @@
 import java.util.Collections;
 import java.util.Random;
 import java.util.TreeSet;
+import java.util.logging.Level;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -43,21 +44,20 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 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.util.AbstractBTreeTest;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.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.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 
 public class RangeSearchCursorTest extends AbstractBTreeTest {
 	// Declare fields
@@ -80,14 +80,16 @@
 
 	@Test
 	public void uniqueIndexTest() throws Exception {
-	    LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
+        }
 
 		// declare keys
 		int keyFieldCount = 1;
 		IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
 		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 
-		MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
 		ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
 	            tupleWriterFactory);
@@ -106,7 +108,7 @@
 		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
 	    ArrayTupleReference tuple = new ArrayTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+	    ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
 		// generate keys
 		int numKeys = 50;
@@ -128,7 +130,7 @@
 			tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
 
 			try {
-				btree.insert(tuple, insertOpCtx);
+			    indexAccessor.insert(tuple);
 			} catch (BTreeException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
@@ -165,7 +167,9 @@
 
 	@Test
 	public void nonUniqueIndexTest() throws Exception {
-	    LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
+        }
 
 		// declare keys
 		int keyFieldCount = 2;
@@ -173,7 +177,7 @@
 		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 		cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
 
-		MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
                 tupleWriterFactory);
@@ -192,7 +196,7 @@
 		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
 		// generate keys
 		int numKeys = 50;
@@ -211,7 +215,7 @@
             tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
 
 			try {
-				btree.insert(tuple, insertOpCtx);
+			    indexAccessor.insert(tuple);
 			} catch (BTreeException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
@@ -248,7 +252,9 @@
 
 	@Test
 	public void nonUniqueFieldPrefixIndexTest() throws Exception {
-	    LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
+        }
 
 		// declare keys
 		int keyFieldCount = 2;
@@ -256,7 +262,7 @@
 		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 		cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE	;			
 
-		MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
         ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
                 tupleWriterFactory);
@@ -275,7 +281,7 @@
 		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
 
-		BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
 
 		// generate keys
 		int numKeys = 50;
@@ -294,7 +300,7 @@
             tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
 
 			try {
-				btree.insert(tuple, insertOpCtx);
+			    indexAccessor.insert(tuple);
 			} catch (BTreeException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
@@ -409,8 +415,8 @@
 				RangePredicate rangePred = createRangePredicate(lowKey,
 						highKey, isForward, lowKeyInclusive, highKeyInclusive,
 						btree.getMultiComparator());
-				BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
-				btree.search(rangeCursor, rangePred, searchOpCtx);
+				ITreeIndexAccessor indexAccessor = btree.createAccessor();
+				indexAccessor.search(rangeCursor, rangePred);
 
 				try {
 					while (rangeCursor.hasNext()) {
@@ -448,32 +454,35 @@
 						else
 							u = ')';
 
-						LOGGER.info("RANGE: " + l + " " + lowKey + " , "
-								+ highKey + " " + u);
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info("RANGE: " + l + " " + lowKey + " , " + highKey + " " + u);
+                        }
 						StringBuilder strBuilder = new StringBuilder();
 						for (Integer r : expectedResults) {
 							strBuilder.append(r + " ");
 						}
-						LOGGER.info(strBuilder.toString());
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info(strBuilder.toString());
+                        }
 					}
 				}
 
 				if (results.size() == expectedResults.size()) {
 					for (int k = 0; k < results.size(); k++) {
 						if (!results.get(k).equals(expectedResults.get(k))) {
-						    LOGGER.info("DIFFERENT RESULTS AT: i=" + i
-									+ " j=" + j + " k=" + k);
-						    LOGGER.info(results.get(k) + " "
-									+ expectedResults.get(k));
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info("DIFFERENT RESULTS AT: i=" + i + " j=" + j + " k=" + k);
+                                LOGGER.info(results.get(k) + " " + expectedResults.get(k));
+                            }
 							return false;
 						}
 					}
 				} else {
-				    LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i
-							+ " j=" + j);
-				    LOGGER.info("RESULTS: " + results.size());
-				    LOGGER.info("EXPECTED RESULTS: "
-							+ expectedResults.size());
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i + " j=" + j);
+                        LOGGER.info("RESULTS: " + results.size());
+                        LOGGER.info("EXPECTED RESULTS: " + expectedResults.size());
+                    }
 					return false;
 				}
 			}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index 7e4b2e3..ac4133d 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -19,6 +19,7 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
+import java.util.logging.Level;
 
 import org.junit.Test;
 
@@ -85,7 +86,9 @@
         private void pinRandomPage() {
             int pageId = Math.abs(rnd.nextInt() % maxPages);
 
-            LOGGER.info(workerId + " PINNING PAGE: " + pageId);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info(workerId + " PINNING PAGE: " + pageId);
+            }
 
             try {
                 ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
@@ -99,14 +102,18 @@
                         break;
 
                     case FTA_READONLY: {
-                        LOGGER.info(workerId + " S LATCHING: " + pageId);
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info(workerId + " S LATCHING: " + pageId);
+                        }
                         page.acquireReadLatch();
                         latch = LatchType.LATCH_S;
                     }
                         break;
 
                     case FTA_WRITEONLY: {
-                        LOGGER.info(workerId + " X LATCHING: " + pageId);
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info(workerId + " X LATCHING: " + pageId);
+                        }
                         page.acquireWriteLatch();
                         latch = LatchType.LATCH_X;
                     }
@@ -114,11 +121,15 @@
 
                     case FTA_MIXED: {
                         if (rnd.nextInt() % 2 == 0) {
-                            LOGGER.info(workerId + " S LATCHING: " + pageId);
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info(workerId + " S LATCHING: " + pageId);
+                            }
                             page.acquireReadLatch();
                             latch = LatchType.LATCH_S;
                         } else {
-                            LOGGER.info(workerId + " X LATCHING: " + pageId);
+                            if (LOGGER.isLoggable(Level.INFO)) {
+                                LOGGER.info(workerId + " X LATCHING: " + pageId);
+                            }
                             page.acquireWriteLatch();
                             latch = LatchType.LATCH_X;
                         }
@@ -141,14 +152,20 @@
 
                 if (plPage.latch != null) {
                     if (plPage.latch == LatchType.LATCH_S) {
-                        LOGGER.info(workerId + " S UNLATCHING: " + plPage.pageId);
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info(workerId + " S UNLATCHING: " + plPage.pageId);
+                        }
                         plPage.page.releaseReadLatch();
                     } else {
-                        LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
+                        if (LOGGER.isLoggable(Level.INFO)) {
+                            LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
+                        }
                         plPage.page.releaseWriteLatch();
                     }
                 }
-                LOGGER.info(workerId + " UNPINNING PAGE: " + plPage.pageId);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(workerId + " UNPINNING PAGE: " + plPage.pageId);
+                }
 
                 bufferCache.unpin(plPage.page);
                 pinnedPages.remove(index);
@@ -158,7 +175,9 @@
         }
 
         private void openFile() {
-            LOGGER.info(workerId + " OPENING FILE: " + fileId);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info(workerId + " OPENING FILE: " + fileId);
+            }
             try {
                 bufferCache.openFile(fileId);
                 fileIsOpen = true;
@@ -168,7 +187,9 @@
         }
 
         private void closeFile() {
-            LOGGER.info(workerId + " CLOSING FILE: " + fileId);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info(workerId + " CLOSING FILE: " + fileId);
+            }
             try {
                 bufferCache.closeFile(fileId);
                 fileIsOpen = false;
@@ -185,7 +206,9 @@
             while (loopCount < maxLoopCount) {
                 loopCount++;
 
-                LOGGER.info(workerId + " LOOP: " + loopCount + "/" + maxLoopCount);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info(workerId + " LOOP: " + loopCount + "/" + maxLoopCount);
+                }
 
                 if (fileIsOpen) {
 
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
index dff1e9c..9630a1b 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
@@ -34,7 +34,7 @@
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
     protected final static String tmpDir = System.getProperty("java.io.tmpdir");
     protected final static String sep = System.getProperty("file.separator");
-    protected String fileName;    
+    protected String fileName;   
     
     @Before
     public void setUp() throws HyracksDataException {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
index c56f1b6..f1b03c1 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -23,39 +23,40 @@
 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.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
 @SuppressWarnings("rawtypes")
-public final class BTreeTestContext {    
+public final class BTreeTestContext {
     public final ISerializerDeserializer[] fieldSerdes;
     public final IBufferCache bufferCache;
     public final BTree btree;
     public final IBTreeLeafFrame leafFrame;
     public final IBTreeInteriorFrame interiorFrame;
-    public final ITreeIndexMetaDataFrame metaFrame;    
+    public final ITreeIndexMetaDataFrame metaFrame;
     public final ArrayTupleBuilder tupleBuilder;
     public final ArrayTupleReference tuple = new ArrayTupleReference();
     public final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
-    public final BTreeOpContext opCtx;
-    
-    public BTreeTestContext(IBufferCache bufferCache, ISerializerDeserializer[] fieldSerdes, BTree btree, IBTreeLeafFrame leafFrame,
-            IBTreeInteriorFrame interiorFrame, ITreeIndexMetaDataFrame metaFrame, BTreeOpContext opCtx) {
+    public final ITreeIndexAccessor indexAccessor;
+
+    public BTreeTestContext(IBufferCache bufferCache, ISerializerDeserializer[] fieldSerdes, BTree btree,
+            IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, ITreeIndexMetaDataFrame metaFrame,
+            ITreeIndexAccessor indexAccessor) {
         this.bufferCache = bufferCache;
         this.fieldSerdes = fieldSerdes;
         this.btree = btree;
         this.leafFrame = leafFrame;
         this.interiorFrame = interiorFrame;
         this.metaFrame = metaFrame;
-        this.opCtx = opCtx;
+        this.indexAccessor = indexAccessor;
         this.tupleBuilder = new ArrayTupleBuilder(fieldSerdes.length);
     }
-    
+
     public int getFieldCount() {
         return fieldSerdes.length;
     }
-    
+
     public int getKeyFieldCount() {
         return btree.getMultiComparator().getKeyFieldCount();
     }
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
index 8e7a43b..8545d14 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
@@ -10,6 +10,7 @@
 import java.util.NavigableSet;
 import java.util.Random;
 import java.util.TreeSet;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -28,15 +29,15 @@
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 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.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 
@@ -51,13 +52,12 @@
         BTree btree = BTreeUtils.createBTree(bufferCache, btreeFileId, typeTraits, cmps, leafType);
         btree.create(btreeFileId);
         btree.open(btreeFileId);
-        // Set an arbitrary index op in the context. Will be reset anyway.
-        BTreeOpContext opCtx = btree.createOpContext(IndexOp.SEARCH);
+        ITreeIndexAccessor indexAccessor = btree.createAccessor();
         
         IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
         IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) btree.getInteriorFrameFactory().createFrame();
         ITreeIndexMetaDataFrame metaFrame = btree.getFreePageManager().getMetaDataFrameFactory().createFrame();
-        BTreeTestContext testCtx = new BTreeTestContext(bufferCache, fieldSerdes, btree, leafFrame, interiorFrame, metaFrame, opCtx);
+        BTreeTestContext testCtx = new BTreeTestContext(bufferCache, fieldSerdes, btree, leafFrame, interiorFrame, metaFrame, indexAccessor);
         return testCtx;
     }
     
@@ -77,8 +77,8 @@
     @SuppressWarnings("unchecked")
     private static CheckTuple createCheckTupleFromTuple(ITupleReference tuple, ISerializerDeserializer[] fieldSerdes, int numKeys) throws HyracksDataException {
         CheckTuple checkTuple = new CheckTuple(fieldSerdes.length, numKeys);
-        int numFields = Math.min(fieldSerdes.length, tuple.getFieldCount());
-        for (int i = 0; i < numFields; i++) {
+        int fieldCount = Math.min(fieldSerdes.length, tuple.getFieldCount());
+        for (int i = 0; i < fieldCount; i++) {
             ByteArrayInputStream inStream = new ByteArrayInputStream(
                     tuple.getFieldData(i), tuple.getFieldStart(i),
                     tuple.getFieldLength(i));
@@ -102,11 +102,12 @@
     }
     
     public static void checkOrderedScan(BTreeTestContext testCtx) throws Exception {
-        LOGGER.info("Testing Ordered Scan.");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Testing Ordered Scan.");
+        }
         ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
         RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
-        testCtx.opCtx.reset(IndexOp.SEARCH);
-        testCtx.btree.search(scanCursor, nullPred, testCtx.opCtx);
+        testCtx.indexAccessor.search(scanCursor, nullPred);
         Iterator<CheckTuple> checkIter = testCtx.checkTuples.iterator();
         int actualCount = 0;
         try {
@@ -129,10 +130,11 @@
     }
     
     public static void checkDiskOrderScan(BTreeTestContext testCtx) throws Exception {
-        LOGGER.info("Testing Disk-Order Scan.");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Testing Disk-Order Scan.");
+        }
         ITreeIndexCursor diskOrderCursor = new TreeDiskOrderScanCursor(testCtx.leafFrame);
-        testCtx.opCtx.reset(IndexOp.DISKORDERSCAN);
-        testCtx.btree.diskOrderScan(diskOrderCursor, testCtx.opCtx);
+        testCtx.indexAccessor.diskOrderScan(diskOrderCursor);
         int actualCount = 0;        
         try {
             while (diskOrderCursor.hasNext()) {
@@ -156,13 +158,14 @@
     }
     
     public static void checkRangeSearch(BTreeTestContext testCtx, ITupleReference lowKey, ITupleReference highKey, boolean lowKeyInclusive, boolean highKeyInclusive) throws Exception {
-        LOGGER.info("Testing Range Search.");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Testing Range Search.");
+        }
         MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), lowKey);
         MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), highKey);
         ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp);
-        testCtx.opCtx.reset(IndexOp.SEARCH);
-        testCtx.btree.search(searchCursor, rangePred, testCtx.opCtx);
+        testCtx.indexAccessor.search(searchCursor, rangePred);
         // Get the subset of elements from the expected set within given key range.
         CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, testCtx.fieldSerdes, lowKeyCmp.getKeyFieldCount());
         CheckTuple highKeyCheck = createCheckTupleFromTuple(highKey, testCtx.fieldSerdes, highKeyCmp.getKeyFieldCount());
@@ -197,14 +200,15 @@
     }
     
     public static void checkPointSearches(BTreeTestContext testCtx) throws Exception {
-        LOGGER.info("Testing Point Searches On All Expected Keys.");        
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Testing Point Searches On All Expected Keys.");
+        }
         ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
         
         ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
         ArrayTupleReference lowKey = new ArrayTupleReference();
         ArrayTupleBuilder highKeyBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
         ArrayTupleReference highKey = new ArrayTupleReference();
-        testCtx.opCtx.reset(IndexOp.SEARCH);
         RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, null, null);
 
         // Iterate through expected tuples, and perform a point search in the BTree to verify the tuple can be reached.
@@ -219,7 +223,7 @@
             rangePred.setLowKeyComparator(lowKeyCmp);
             rangePred.setHighKeyComparator(highKeyCmp);
             
-            testCtx.btree.search(searchCursor, rangePred, testCtx.opCtx);
+            testCtx.indexAccessor.search(searchCursor, rangePred);
             
             try {
                 // We expect exactly one answer.
@@ -267,11 +271,8 @@
     }
     
     public static void insertIntTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
-        int numFields = testCtx.getFieldCount();
+        int fieldCount = testCtx.getFieldCount();
         int numKeyFields = testCtx.getKeyFieldCount();
-        
-        testCtx.opCtx.reset(IndexOp.INSERT);
-        
         int[] tupleValues = new int[testCtx.getFieldCount()];
         // Scale range of values according to number of keys. 
         // For example, for 2 keys we want the square root of numTuples, for 3 keys the cube root of numTuples, etc.        
@@ -282,17 +283,19 @@
                 tupleValues[j] = rnd.nextInt() % maxValue;
             }
             // Set values.
-            for (int j = numKeyFields; j < numFields; j++) {
+            for (int j = numKeyFields; j < fieldCount; j++) {
                 tupleValues[j] = j;
             }
             TupleUtils.createIntegerTuple(testCtx.tupleBuilder, testCtx.tuple, tupleValues);
-            if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
-                LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+                    LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+                }
             }
             try {
-                testCtx.btree.insert(testCtx.tuple, testCtx.opCtx);
+                testCtx.indexAccessor.insert(testCtx.tuple);
                 // Set expected values. Do this only after insertion succeeds because we ignore duplicate keys.
-                CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(numFields, numKeyFields);
+                CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldCount, numKeyFields);
                 for(int v : tupleValues) {
                     checkTuple.add(v);
                 }
@@ -304,15 +307,14 @@
     }
     
     public static void insertStringTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
-        int numFields = testCtx.getFieldCount();
+        int fieldCount = testCtx.getFieldCount();
         int numKeyFields = testCtx.getKeyFieldCount();
-        
-        testCtx.opCtx.reset(IndexOp.INSERT);
-
-        Object[] tupleValues = new Object[numFields];
+        Object[] tupleValues = new Object[fieldCount];
         for (int i = 0; i < numTuples; i++) {
-            if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
-                LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+                    LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+                }
             }
             // Set keys.
             for (int j = 0; j < numKeyFields; j++) {
@@ -320,14 +322,14 @@
                 tupleValues[j] = getRandomString(length, rnd);
             }
             // Set values.
-            for (int j = numKeyFields; j < numFields; j++) {
+            for (int j = numKeyFields; j < fieldCount; j++) {
                 tupleValues[j] = getRandomString(5, rnd);
             }
             TupleUtils.createTuple(testCtx.tupleBuilder, testCtx.tuple, testCtx.fieldSerdes, tupleValues);
             try {
-                testCtx.btree.insert(testCtx.tuple, testCtx.opCtx);
+                testCtx.indexAccessor.insert(testCtx.tuple);
                 // Set expected values. Do this only after insertion succeeds because we ignore duplicate keys.
-                CheckTuple<String> checkTuple = new CheckTuple<String>(numFields, numKeyFields);
+                CheckTuple<String> checkTuple = new CheckTuple<String>(fieldCount, numKeyFields);
                 for(Object v : tupleValues) {
                     checkTuple.add((String)v);
                 }
@@ -339,7 +341,7 @@
     }
 
     public static void bulkLoadIntTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
-        int numFields = testCtx.getFieldCount();
+        int fieldCount = testCtx.getFieldCount();
         int numKeyFields = testCtx.getKeyFieldCount();
         int[] tupleValues = new int[testCtx.getFieldCount()];
         int maxValue = (int)Math.ceil(Math.pow(numTuples, 1.0/(double)numKeyFields));
@@ -349,12 +351,12 @@
                 tupleValues[j] = rnd.nextInt() % maxValue;
             }
             // Set values.
-            for (int j = numKeyFields; j < numFields; j++) {
+            for (int j = numKeyFields; j < fieldCount; j++) {
                 tupleValues[j] = j;
             }
             
             // Set expected values. We also use these as the pre-sorted stream for bulk loading.
-            CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(numFields, numKeyFields);
+            CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldCount, numKeyFields);
             for(int v : tupleValues) {
                 checkTuple.add(v);
             }            
@@ -365,9 +367,9 @@
     }
     
     public static void bulkLoadStringTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
-        int numFields = testCtx.getFieldCount();
+        int fieldCount = testCtx.getFieldCount();
         int numKeyFields = testCtx.getKeyFieldCount();
-        String[] tupleValues = new String[numFields];
+        String[] tupleValues = new String[fieldCount];
         for (int i = 0; i < numTuples; i++) {
             // Set keys.
             for (int j = 0; j < numKeyFields; j++) {
@@ -375,11 +377,11 @@
                 tupleValues[j] = getRandomString(length, rnd);
             }
             // Set values.
-            for (int j = numKeyFields; j < numFields; j++) {
+            for (int j = numKeyFields; j < fieldCount; j++) {
                 tupleValues[j] = getRandomString(5, rnd);
             }
             // Set expected values. We also use these as the pre-sorted stream for bulk loading.
-            CheckTuple<String> checkTuple = new CheckTuple<String>(numFields, numKeyFields);
+            CheckTuple<String> checkTuple = new CheckTuple<String>(fieldCount, numKeyFields);
             for(String v : tupleValues) {
                 checkTuple.add(v);
             }            
@@ -389,16 +391,18 @@
         bulkLoadCheckTuples(testCtx, numTuples);
     }
     
-    private static void bulkLoadCheckTuples(BTreeTestContext testCtx, int numTuples) throws HyracksDataException, TreeIndexException {
-        int numFields = testCtx.getFieldCount();
-        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(numFields);
+    private static void bulkLoadCheckTuples(BTreeTestContext testCtx, int numTuples) throws HyracksDataException, TreeIndexException, PageAllocationException {
+        int fieldCount = testCtx.getFieldCount();
+        ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference tuple = new ArrayTupleReference();
         // Perform bulk load.
         IIndexBulkLoadContext bulkLoadCtx = testCtx.btree.beginBulkLoad(0.7f);
         int c = 1;
         for (CheckTuple checkTuple : testCtx.checkTuples) {
-            if (c % (numTuples / 10) == 0) {
-                LOGGER.info("Bulk Loading Tuple " + c + "/" + numTuples);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if (c % (numTuples / 10) == 0) {
+                    LOGGER.info("Bulk Loading Tuple " + c + "/" + numTuples);
+                }
             }
             createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, testCtx.fieldSerdes);
             testCtx.btree.bulkLoadAddTuple(tuple, bulkLoadCtx);
@@ -411,7 +415,6 @@
         ArrayTupleBuilder deleteTupleBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
         ArrayTupleReference deleteTuple = new ArrayTupleReference();
         int numCheckTuples = testCtx.checkTuples.size();        
-        testCtx.opCtx.reset(IndexOp.DELETE);
         // Copy CheckTuple references into array, so we can randomly pick from there.
         CheckTuple[] checkTuples = new CheckTuple[numCheckTuples];
         int idx = 0;
@@ -419,13 +422,15 @@
             checkTuples[idx++] = checkTuple;
         }
         for (int i = 0; i < numTuples && numCheckTuples > 0; i++) {
-            if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
-                LOGGER.info("Deleting Tuple " + (i + 1) + "/" + numTuples);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+                    LOGGER.info("Deleting Tuple " + (i + 1) + "/" + numTuples);
+                }
             }
             int checkTupleIdx = Math.abs(rnd.nextInt() % numCheckTuples);
             CheckTuple checkTuple = checkTuples[checkTupleIdx];            
             createTupleFromCheckTuple(checkTuple, deleteTupleBuilder, deleteTuple, testCtx.fieldSerdes);          
-            testCtx.btree.delete(deleteTuple, testCtx.opCtx);
+            testCtx.indexAccessor.delete(deleteTuple);
             
             // Remove check tuple from expected results.
             testCtx.checkTuples.remove(checkTuple);
@@ -449,7 +454,6 @@
         ArrayTupleBuilder updateTupleBuilder = new ArrayTupleBuilder(fieldCount);
         ArrayTupleReference updateTuple = new ArrayTupleReference();
         int numCheckTuples = testCtx.checkTuples.size();
-        testCtx.opCtx.reset(IndexOp.UPDATE);
         // Copy CheckTuple references into array, so we can randomly pick from there.
         CheckTuple[] checkTuples = new CheckTuple[numCheckTuples];
         int idx = 0;
@@ -457,8 +461,10 @@
             checkTuples[idx++] = checkTuple;
         }
         for (int i = 0; i < numTuples && numCheckTuples > 0; i++) {
-            if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
-                LOGGER.info("Updating Tuple " + (i + 1) + "/" + numTuples);
+            if (LOGGER.isLoggable(Level.INFO)) {
+                if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+                    LOGGER.info("Updating Tuple " + (i + 1) + "/" + numTuples);
+                }
             }
             int checkTupleIdx = Math.abs(rnd.nextInt() % numCheckTuples);
             CheckTuple checkTuple = checkTuples[checkTupleIdx];
@@ -469,7 +475,7 @@
             }
             
             createTupleFromCheckTuple(checkTuple, updateTupleBuilder, updateTuple, testCtx.fieldSerdes);            
-            testCtx.btree.update(updateTuple, testCtx.opCtx);
+            testCtx.indexAccessor.update(updateTuple);
             
             // Swap with last "valid" CheckTuple.
             CheckTuple tmp = checkTuples[numCheckTuples - 1];
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index fa1833a..0f3d944 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package edu.uci.ics.hyracks.storage.am.invertedindex;
 
 import java.io.DataOutput;
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 86b3107..0d55dec 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
@@ -48,10 +48,10 @@
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
 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.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
@@ -59,7 +59,6 @@
 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.tuples.TypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -241,7 +240,7 @@
         IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
         tokenAccessor.reset(frame);
 
-        BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH);
+        ITreeIndexAccessor btreeAccessor = invIndex.getBTree().createAccessor();
 
         // verify created inverted lists one-by-one
         for (int i = 0; i < tokens.size(); i++) {
@@ -256,7 +255,7 @@
 
             searchKey.reset(tokenAccessor, 0);
 
-            invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
+            invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
 
             invListCursor.pinPagesSync();
             int checkIndex = 0;
@@ -291,7 +290,7 @@
 
             searchKey.reset(tokenAccessor, 0);
 
-            invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
+            invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
 
             invListCursor.pinPagesSync();
             Assert.assertEquals(invListCursor.hasNext(), false);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
index e6dba19..1299fcb 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
@@ -17,6 +17,7 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.logging.Level;
 
 import org.junit.Assert;
 import org.junit.Before;
@@ -27,6 +28,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -66,7 +68,7 @@
 		loadData();
 	}
 
-	public void loadData() throws HyracksDataException, TreeIndexException {
+	public void loadData() throws HyracksDataException, TreeIndexException, PageAllocationException {
 		tokens.add("compilers");
 		tokens.add("computer");
 		tokens.add("databases");
@@ -212,7 +214,10 @@
 				totalTime += timeEnd - timeStart;
 			}
 			double avgTime = totalTime / (double) repeats;
-			LOGGER.info(i + ": " + "\"" + queryString + "\": " + avgTime + "ms");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info(i + ": " + "\"" + queryString + "\": " + avgTime
+						+ "ms");
+			}
 
 			if (!panic) {
 
@@ -233,12 +238,16 @@
 				}
 
 				if (expectedResults.size() != checkIndex) {
-					LOGGER.info("CHECKING");
+					if (LOGGER.isLoggable(Level.INFO)) {
+						LOGGER.info("CHECKING");
+					}
 					StringBuilder expectedStrBuilder = new StringBuilder();
 					for (Integer x : expectedResults) {
 						expectedStrBuilder.append(x + " ");
 					}
-					LOGGER.info(expectedStrBuilder.toString());
+					if (LOGGER.isLoggable(Level.INFO)) {
+						LOGGER.info(expectedStrBuilder.toString());
+					}
 				}
 
 				Assert.assertEquals(expectedResults.size(), checkIndex);
@@ -266,27 +275,39 @@
 	public void jaccardKeywordQueryTest() throws Exception {
 		JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
 
-		LOGGER.info("JACCARD: " + 1.0f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 1.0f);
+		}
 		searchModifier.setJaccThresh(1.0f);
 		runQueries(searchModifier, 50);
 
-		LOGGER.info("JACCARD: " + 0.9f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.9f);
+		}
 		searchModifier.setJaccThresh(0.9f);
 		runQueries(searchModifier, 50);
 
-		LOGGER.info("JACCARD: " + 0.8f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.8f);
+		}
 		searchModifier.setJaccThresh(0.8f);
 		runQueries(searchModifier, 50);
 
-		LOGGER.info("JACCARD: " + 0.7f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.7f);
+		}
 		searchModifier.setJaccThresh(0.7f);
 		runQueries(searchModifier, 50);
 
-		LOGGER.info("JACCARD: " + 0.6f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.6f);
+		}
 		searchModifier.setJaccThresh(0.6f);
 		runQueries(searchModifier, 50);
 
-		LOGGER.info("JACCARD: " + 0.5f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.5f);
+		}
 		searchModifier.setJaccThresh(0.5f);
 		runQueries(searchModifier, 50);
 	}
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 91fb924..eb2b39c 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
@@ -1,3 +1,18 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package edu.uci.ics.hyracks.storage.am.invertedindex;
 
 import java.io.DataOutputStream;
@@ -5,6 +20,7 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.logging.Level;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -14,6 +30,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
 import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -113,7 +130,7 @@
 		}
 	}
 
-	public void loadData() throws IOException, TreeIndexException {
+	public void loadData() throws IOException, TreeIndexException, PageAllocationException {
 		List<TokenIdPair> pairs = new ArrayList<TokenIdPair>();
 		// generate pairs for subsequent sorting and bulk-loading
 		int id = 0;
@@ -215,7 +232,9 @@
 			}
 			// remove trailing newline
 			strBuilder.deleteCharAt(strBuilder.length() - 1);
-			LOGGER.info(strBuilder.toString());
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info(strBuilder.toString());
+			}
 		}
 	}
 
@@ -239,15 +258,21 @@
 	public void jaccardQueryTest() throws Exception {
 		JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
 
-		LOGGER.info("JACCARD: " + 0.9f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.9f);
+		}
 		searchModifier.setJaccThresh(0.9f);
 		runQueries(searchModifier, 5);
 
-		LOGGER.info("JACCARD: " + 0.8f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.8f);
+		}
 		searchModifier.setJaccThresh(0.8f);
 		runQueries(searchModifier, 5);
 
-		LOGGER.info("JACCARD: " + 0.7f);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("JACCARD: " + 0.7f);
+		}
 		searchModifier.setJaccThresh(0.7f);
 		runQueries(searchModifier, 5);
 	}
@@ -262,15 +287,21 @@
 		EditDistanceSearchModifier searchModifier = new EditDistanceSearchModifier(
 				3, 0);
 
-		LOGGER.info("EDIT DISTANCE: " + 1);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("EDIT DISTANCE: " + 1);
+		}
 		searchModifier.setEdThresh(1);
 		runQueries(searchModifier, 5);
 
-		LOGGER.info("EDIT DISTANCE: " + 2);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("EDIT DISTANCE: " + 2);
+		}
 		searchModifier.setEdThresh(2);
 		runQueries(searchModifier, 5);
 
-		LOGGER.info("EDIT DISTANCE: " + 3);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("EDIT DISTANCE: " + 3);
+		}
 		searchModifier.setEdThresh(3);
 		runQueries(searchModifier, 5);
 	}
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java
index 25c5370..8f619a4 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java
@@ -18,11 +18,13 @@
 import java.io.File;
 import java.text.SimpleDateFormat;
 import java.util.Date;
+import java.util.logging.Logger;
 
 import org.junit.AfterClass;
 
 public abstract class AbstractRTreeTest {
 
+	protected static final Logger LOGGER = Logger.getLogger(AbstractRTreeTest.class.getName());
 	protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
 			"ddMMyy-hhmmssSS");
 	protected final static String tmpDir = System.getProperty("java.io.tmpdir");
@@ -30,10 +32,6 @@
 	protected final static String fileName = tmpDir + sep
 			+ simpleDateFormat.format(new Date());
 
-	protected void print(String str) {
-		System.err.print(str);
-	}
-
 	@AfterClass
 	public static void cleanup() throws Exception {
 		File f = new File(fileName);
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 ce3bd3c..faf1e88 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
@@ -19,6 +19,7 @@
 import java.io.File;
 import java.nio.ByteBuffer;
 import java.util.Random;
+import java.util.logging.Level;
 
 import org.junit.Test;
 
@@ -42,6 +43,7 @@
 import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 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;
@@ -49,7 +51,6 @@
 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.impls.TreeDiskOrderScanCursor;
-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.util.TreeIndexStats;
 import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
@@ -57,7 +58,6 @@
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -155,7 +155,7 @@
 		accessor.reset(hyracksFrame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
 		Random rnd = new Random();
 		rnd.setSeed(50);
@@ -199,38 +199,42 @@
 
 			tuple.reset(accessor, 0);
 
-			if (i % 1000 == 0) {
-				print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-						+ Math.max(p1y, p2y) + "\n");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				if (i % 1000 == 0) {
+					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x)
+							+ " " + Math.min(p1y, p2y) + " "
+							+ Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+				}
 			}
 
 			try {
-				rtree.insert(tuple, insertOpCtx);
+				indexAccessor.insert(tuple);
 			} catch (TreeIndexException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
 			}
 		}
 
-		// rtree.printTree(leafFrame, interiorFrame, recDescSers);
-		// System.err.println();
-
 		String rtreeStats = rtree.printStats();
-		print(rtreeStats);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(rtreeStats);
+		}
 
 		// disk-order scan
-		print("DISK-ORDER SCAN:\n");
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("DISK-ORDER SCAN:");
+		}
 		TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
 				leafFrame);
-		RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
-		rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+		indexAccessor.diskOrderScan(diskOrderCursor);
 		try {
 			while (diskOrderCursor.hasNext()) {
 				diskOrderCursor.next();
 				ITupleReference frameTuple = diskOrderCursor.getTuple();
 				String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-				print(rec + "\n");
+				if (LOGGER.isLoggable(Level.INFO)) {
+					LOGGER.info(rec);
+				}
 			}
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -243,7 +247,9 @@
 		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
 				interiorFrame, metaFrame);
 		String string = stats.toString();
-		System.err.println(string);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(string);
+		}
 
 		rtree.close();
 		bufferCache.closeFile(fileId);
@@ -334,7 +340,7 @@
 		accessor.reset(hyracksFrame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
 		Random rnd = new Random();
 		rnd.setSeed(50);
@@ -376,27 +382,27 @@
 
 			tuple.reset(accessor, 0);
 
-			if (i % 1000 == 0) {
-				print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-						+ Math.max(p1y, p2y) + "\n");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				if (i % 1000 == 0) {
+					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+							+ " " + Math.max(p1y, p2y));
+				}
 			}
 
 			try {
-				rtree.insert(tuple, insertOpCtx);
+				indexAccessor.insert(tuple);
 			} catch (TreeIndexException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
 			}
 		}
 
-		// rtree.printTree(leafFrame, interiorFrame, recDescSers);
-		// System.err.println();
-
 		String rtreeStats = rtree.printStats();
-		print(rtreeStats);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(rtreeStats);
+		}
 
-		RTreeOpContext deleteOpCtx = rtree.createOpContext(IndexOp.DELETE);
 		rnd.setSeed(50);
 		for (int i = 0; i < 5000; i++) {
 
@@ -434,16 +440,17 @@
 					tb.getSize());
 
 			tuple.reset(accessor, 0);
-
-			if (i % 1000 == 0) {
-				print("DELETING " + i + " " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-						+ Math.max(p1y, p2y) + "\n");
+			
+			if (LOGGER.isLoggable(Level.INFO)) {
+				if (i % 1000 == 0) {
+					LOGGER.info("DELETING " + i + " " + Math.min(p1x, p2x) + " "
+							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+							+ " " + Math.max(p1y, p2y));
+				}
 			}
 
 			try {
-				rtree.delete(tuple, deleteOpCtx);
-
+				indexAccessor.delete(tuple);
 			} catch (TreeIndexException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
@@ -455,7 +462,9 @@
 		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
 				interiorFrame, metaFrame);
 		String string = stats.toString();
-		System.err.println(string);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(string);
+		}
 
 		rtree.close();
 		bufferCache.closeFile(fileId);
@@ -552,7 +561,7 @@
 		accessor.reset(hyracksFrame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
 		Random rnd = new Random();
 		rnd.setSeed(50);
@@ -602,39 +611,43 @@
 
 			tuple.reset(accessor, 0);
 
-			if (i % 1000 == 0) {
-				print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.min(p1z, p2z) + " "
-						+ " " + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y)
-						+ " " + Math.max(p1z, p2z) + "\n");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				if (i % 1000 == 0) {
+					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+							+ Math.min(p1y, p2y) + " " + Math.min(p1z, p2z)
+							+ " " + " " + Math.max(p1x, p2x) + " "
+							+ Math.max(p1y, p2y) + " " + Math.max(p1z, p2z));
+				}
 			}
 
 			try {
-				rtree.insert(tuple, insertOpCtx);
+				indexAccessor.insert(tuple);
 			} catch (TreeIndexException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
 			}
 		}
 
-		// rtree.printTree(leafFrame, interiorFrame, recDescSers);
-		// System.err.println();
-
 		String rtreeStats = rtree.printStats();
-		print(rtreeStats);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(rtreeStats);
+		}
 
 		// disk-order scan
-		print("DISK-ORDER SCAN:\n");
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("DISK-ORDER SCAN:");
+		}
 		TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
 				leafFrame);
-		RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
-		rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+		indexAccessor.diskOrderScan(diskOrderCursor);
 		try {
 			while (diskOrderCursor.hasNext()) {
 				diskOrderCursor.next();
 				ITupleReference frameTuple = diskOrderCursor.getTuple();
 				String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-				print(rec + "\n");
+				if (LOGGER.isLoggable(Level.INFO)) {
+					LOGGER.info(rec);
+				}
 			}
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -647,7 +660,9 @@
 		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
 				interiorFrame, metaFrame);
 		String string = stats.toString();
-		System.err.println(string);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(string);
+		}
 
 		rtree.close();
 		bufferCache.closeFile(fileId);
@@ -738,7 +753,7 @@
 		accessor.reset(hyracksFrame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
 		Random rnd = new Random();
 		rnd.setSeed(50);
@@ -782,38 +797,42 @@
 
 			tuple.reset(accessor, 0);
 
-			if (i % 1000 == 0) {
-				print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-						+ Math.max(p1y, p2y) + "\n");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				if (i % 1000 == 0) {
+					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+							+ " " + Math.max(p1y, p2y));
+				}
 			}
 
 			try {
-				rtree.insert(tuple, insertOpCtx);
+				indexAccessor.insert(tuple);
 			} catch (TreeIndexException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
 			}
 		}
 
-		// rtree.printTree(leafFrame, interiorFrame, recDescSers);
-		// System.err.println();
-
 		String rtreeStats = rtree.printStats();
-		print(rtreeStats);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(rtreeStats);
+		}
 
 		// disk-order scan
-		print("DISK-ORDER SCAN:\n");
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("DISK-ORDER SCAN:");
+		}
 		TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
 				leafFrame);
-		RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
-		rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+		indexAccessor.diskOrderScan(diskOrderCursor);
 		try {
 			while (diskOrderCursor.hasNext()) {
 				diskOrderCursor.next();
 				ITupleReference frameTuple = diskOrderCursor.getTuple();
 				String rec = TupleUtils.printTuple(frameTuple, recDescSers);
-				print(rec + "\n");
+				if (LOGGER.isLoggable(Level.INFO)) {
+					LOGGER.info(rec);
+				}
 			}
 		} catch (Exception e) {
 			e.printStackTrace();
@@ -826,7 +845,9 @@
 		TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
 				interiorFrame, metaFrame);
 		String string = stats.toString();
-		System.err.println(string);
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info(string);
+		}
 
 		rtree.close();
 		bufferCache.closeFile(fileId);
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 c87f0da..60d0578 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
@@ -23,6 +23,7 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Random;
+import java.util.logging.Level;
 
 import org.junit.Test;
 
@@ -44,6 +45,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
@@ -51,14 +53,12 @@
 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.rtree.api.IRTreeInteriorFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
 import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
@@ -155,7 +155,7 @@
 		accessor.reset(hyracksFrame);
 		FrameTupleReference tuple = new FrameTupleReference();
 
-		RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+		ITreeIndexAccessor indexAccessor = rtree.createAccessor();
 
 		Random rnd = new Random();
 		rnd.setSeed(50);
@@ -190,14 +190,16 @@
 
 			tuple.reset(accessor, 0);
 
-			if (i % 1000 == 0) {
-				print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
-						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-						+ Math.max(p1y, p2y) + "\n");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				if (i % 1000 == 0) {
+					LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+							+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+							+ " " + Math.max(p1y, p2y));
+				}
 			}
 
 			try {
-				rtree.insert(tuple, insertOpCtx);
+				indexAccessor.insert(tuple);
 			} catch (TreeIndexException e) {
 			} catch (Exception e) {
 				e.printStackTrace();
@@ -234,16 +236,17 @@
 
 			tuple.reset(accessor, 0);
 
-			print(i + " Searching for: " + Math.min(p1x, p2x) + " "
-					+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
-					+ Math.max(p1y, p2y) + "\n");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info(i + " Searching for: " + Math.min(p1x, p2x) + " "
+						+ Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
+						+ Math.max(p1y, p2y));
+			}
 
 			ITreeIndexCursor searchCursor = new RTreeSearchCursor(
 					interiorFrame, leafFrame);
 			SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
 
-			RTreeOpContext searchOpCtx = rtree.createOpContext(IndexOp.SEARCH);
-			rtree.search(searchCursor, searchPredicate, searchOpCtx);
+			indexAccessor.search(searchCursor, searchPredicate);
 
 			ArrayList<Integer> results = new ArrayList<Integer>();
 			try {
@@ -264,9 +267,10 @@
 			} finally {
 				searchCursor.close();
 			}
-
-			System.err.println("There are " + results.size()
-					+ " objects that satisfy the query");
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("There are " + results.size()
+						+ " objects that satisfy the query");
+			}
 		}
 
 		rtree.close();
diff --git a/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java
new file mode 100644
index 0000000..6c91cdd
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java
@@ -0,0 +1,180 @@
+package edu.uci.ics.hyracks.storage.common;
+
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IIOManager.FileReadWriteMode;
+import edu.uci.ics.hyracks.api.io.IIOManager.FileSyncMode;
+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;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class BufferCacheRegressionTests {
+	protected static final String tmpDir = System.getProperty("java.io.tmpdir");
+	protected static final String sep = System.getProperty("file.separator");
+
+	protected String fileName = tmpDir + sep + "flushTestFile";
+
+	private static final int PAGE_SIZE = 256;
+	private static final int HYRACKS_FRAME_SIZE = PAGE_SIZE;
+	private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
+	// We want to test the following behavior when reclaiming a file slot in the
+	// buffer cache:
+	// 1. If the file being evicted was deleted, then its dirty pages should be
+	// invalidated, but most not be flushed.
+	// 2. If the file was not deleted, then we must flush its dirty pages.
+	@Test
+	public void testFlushBehaviorOnFileEviction() throws IOException {
+		File f = new File(fileName);
+		if (f.exists()) {
+			f.delete();
+		}
+		flushBehaviorTest(true);
+		flushBehaviorTest(false);
+	}
+
+	private void flushBehaviorTest(boolean deleteFile) throws IOException {
+		TestStorageManagerComponentHolder.init(PAGE_SIZE, 10, 1);
+
+		IBufferCache bufferCache = TestStorageManagerComponentHolder
+				.getBufferCache(ctx);
+		IFileMapProvider fmp = TestStorageManagerComponentHolder
+				.getFileMapProvider(ctx);
+
+		FileReference firstFileRef = new FileReference(new File(fileName));
+		bufferCache.createFile(firstFileRef);
+		int firstFileId = fmp.lookupFileId(firstFileRef);
+		bufferCache.openFile(firstFileId);
+
+		// Fill the first page with known data and make it dirty by write
+		// latching it.
+		ICachedPage writePage = bufferCache.pin(
+				BufferedFileHandle.getDiskPageId(firstFileId, 0), true);
+		writePage.acquireWriteLatch();
+		try {
+			ByteBuffer buf = writePage.getBuffer();
+			for (int i = 0; i < buf.capacity(); i++) {
+				buf.put(Byte.MAX_VALUE);
+			}
+		} finally {
+			writePage.releaseWriteLatch();
+			bufferCache.unpin(writePage);
+		}
+		bufferCache.closeFile(firstFileId);
+		if (deleteFile) {
+			bufferCache.deleteFile(firstFileId);
+		}
+
+		// Create a file with the same name.
+		FileReference secondFileRef = new FileReference(new File(fileName));
+		bufferCache.createFile(secondFileRef);
+		int secondFileId = fmp.lookupFileId(secondFileRef);
+
+		// This open will replace the firstFileRef's slot in the BufferCache,
+		// causing it's pages to be cleaned up. We want to make sure that those
+		// dirty pages are not flushed to the disk, because the file was
+		// declared as deleted, and
+		// somebody might be already using the same filename again (having been
+		// assigned a different fileId).
+		bufferCache.openFile(secondFileId);
+
+		// Manually open the file and inspect it's contents. We cannot simply
+		// ask the BufferCache to pin the page, because it would return the same
+		// physical memory again, and for performance reasons pages are never
+		// reset with 0's.
+		IIOManager ioManager = ctx.getIOManager();
+		FileReference testFileRef = new FileReference(new File(fileName));
+		FileHandle testFileHandle = new FileHandle(testFileRef);
+		testFileHandle.open(FileReadWriteMode.READ_ONLY,
+				FileSyncMode.METADATA_SYNC_DATA_SYNC);
+		ByteBuffer testBuffer = ByteBuffer.allocate(PAGE_SIZE);
+		ioManager.syncRead(testFileHandle, 0, testBuffer);
+		for (int i = 0; i < testBuffer.capacity(); i++) {
+			if (deleteFile) {
+				// We deleted the file. We expect to see a clean buffer.
+				if (testBuffer.get(i) == Byte.MAX_VALUE) {
+					fail("Page 0 of deleted file was fazily flushed in openFile(), "
+							+ "corrupting the data of a newly created file with the same name.");
+				}
+			} else {
+				// We didn't delete the file. We expect to see a buffer full of
+				// Byte.MAX_VALUE.
+				if (testBuffer.get(i) != Byte.MAX_VALUE) {
+					fail("Page 0 of closed file was not flushed when properly, when reclaiming the file slot of fileId 0 in the BufferCache.");
+				}
+			}
+		}
+		testFileHandle.close();
+		bufferCache.closeFile(secondFileId);
+		if (deleteFile) {
+			bufferCache.deleteFile(secondFileId);
+		}
+		bufferCache.close();
+	}
+
+	// Tests the behavior of the BufferCache when more than all pages are
+	// pinned. We expect an exception.
+	@Test
+	public void testPinningAllPages() throws HyracksDataException {
+		int numPages = 10;
+		TestStorageManagerComponentHolder.init(PAGE_SIZE, numPages, 1);
+
+		IBufferCache bufferCache = TestStorageManagerComponentHolder
+				.getBufferCache(ctx);
+		IFileMapProvider fmp = TestStorageManagerComponentHolder
+				.getFileMapProvider(ctx);
+
+		FileReference firstFileRef = new FileReference(new File(fileName));
+		bufferCache.createFile(firstFileRef);
+		int fileId = fmp.lookupFileId(firstFileRef);
+		bufferCache.openFile(fileId);
+
+		// Pin all pages.
+		ICachedPage[] pages = new ICachedPage[numPages];
+		for (int i = 0; i < numPages; ++i) {
+			pages[i] = bufferCache.pin(
+					BufferedFileHandle.getDiskPageId(fileId, i), true);
+		}
+
+		// Try to pin another page. We expect a HyracksDataException.
+		ICachedPage errorPage = null;
+		try {
+			errorPage = bufferCache.pin(
+					BufferedFileHandle.getDiskPageId(fileId, numPages), true);
+		} catch (HyracksDataException e) {
+			// This is the expected outcome.
+			// The BufferCache should still be able to function properly.
+			// Try unpinning all pages.
+			for (int i = 0; i < numPages; ++i) {
+				bufferCache.unpin(pages[i]);
+			}
+			// Now try pinning the page that failed above again.
+			errorPage = bufferCache.pin(
+					BufferedFileHandle.getDiskPageId(fileId, numPages), true);
+			// Unpin it.
+			bufferCache.unpin(errorPage);
+			// Cleanup.
+			bufferCache.closeFile(fileId);
+			bufferCache.close();
+			return;
+		} catch (Exception e) {
+			fail("Expected a HyracksDataException when pinning more pages than available but got another exception: "
+					+ e.getMessage());
+		}
+		fail("Expected a HyracksDataException when pinning more pages than available.");
+	}
+}