Polished inverted index. Added integration test for inverted index bulk load.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@775 123451ca-8445-de46-9d55-352943316053
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 96%
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 c84ef74..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
@@ -28,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 {
@@ -52,9 +52,9 @@
         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);
 
         PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
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..26ac7dc
--- /dev/null
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -0,0 +1,282 @@
+/*
+ * 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.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+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 });
+
+    @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 test() {
+        // TODO: perform inverted index searches.
+    }
+
+    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) {
+        ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+        IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+                tokenFactory);
+        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);
+    }
+}
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 919ba18..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;
@@ -32,6 +31,7 @@
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 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;
@@ -98,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);
 
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/dataflow/TreeIndexOpHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
index 6fea7e5..353b013 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;
 
@@ -74,7 +75,6 @@
         // 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();
@@ -83,20 +83,11 @@
             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);
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/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
index b93f064..4dc655e 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,
+            ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
             ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
-            ITreeIndexOpHelperFactory opHelperFactory, IFileSplitProvider invIndexFileSplitProvider,
-            IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
-            IBinaryComparatorFactory[] invIndexComparatorFactories) {
+            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 b5b1393..1041deb 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
@@ -32,17 +32,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;
     }
 
@@ -50,6 +50,6 @@
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             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 1ab72fe..a250750 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
@@ -27,7 +27,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;
 
@@ -37,29 +36,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, btreeFillFactor,
+                btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider, invListsTypeTraits,
+                invListComparatorFactories);
         this.fieldPermutation = fieldPermutation;
         this.btreeFillFactor = btreeFillFactor;
-        this.invListBuilder = invListBuilder;
     }
 
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
             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 988f073..7df754a 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
@@ -27,10 +27,11 @@
 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;
@@ -46,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);
+            }
         }
     }
 
@@ -94,10 +103,10 @@
         for (int i = 0; i < tupleCount; i++) {
             tuple.reset(accessor, i);
             try {
-				invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
-			} catch (PageAllocationException e) {
-				throw new HyracksDataException(e);
-			}
+                invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
+            } catch (PageAllocationException e) {
+                throw new HyracksDataException(e);
+            }
         }
     }
 
@@ -106,9 +115,9 @@
         try {
             invIndexOpHelper.getInvIndex().endBulkLoad(bulkLoadCtx);
         } catch (PageAllocationException e) {
-        	throw new HyracksDataException(e);
-		} finally {
-            treeIndexOpHelper.deinit();
+            throw new HyracksDataException(e);
+        } finally {
+            btreeOpHelper.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..33b0eef 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,99 +40,63 @@
     private IInvertedIndexOperatorDescriptorHelper opDesc;
     private IHyracksTaskContext ctx;
 
-    private IndexHelperOpenMode mode;
+	public InvertedIndexOpHelper(TreeIndexOpHelper btreeOpHelper,
+			IInvertedIndexOperatorDescriptorHelper opDesc,
+			final IHyracksTaskContext ctx, int partition) {
+		this.btreeOpHelper = btreeOpHelper;
+		this.opDesc = opDesc;
+		this.ctx = ctx;
+		this.partition = partition;
+	}
 
-    public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
-            int partition, IndexHelperOpenMode mode) {
-        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;
-
+        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
+            // 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();
-            }
-        }
-    }
+		// 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);
+		// 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) {
+				// 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);
+			}
+		} finally {
+			invIndexRegistry.unlock();
+		}
+	}
 
     public void deinit() throws HyracksDataException {
         if (invIndexFileId != -1) {
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-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/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 0b4bd62..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
@@ -28,7 +28,6 @@
 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.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;
@@ -39,6 +38,7 @@
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.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);
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 bb15de9..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
@@ -56,7 +56,6 @@
 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;
@@ -71,6 +70,7 @@
 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.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;
@@ -115,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);        
@@ -342,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);        
@@ -545,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);        
@@ -730,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);        
@@ -948,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);        
@@ -1120,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);        
@@ -1283,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);        
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 7834419..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
@@ -47,7 +47,6 @@
 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;
@@ -58,6 +57,7 @@
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
 
 public class RangeSearchCursorTest extends AbstractBTreeTest {
 	// Declare fields
@@ -89,7 +89,7 @@
 		IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
 		cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
 
-		MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+		MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
 
 		ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
 	            tupleWriterFactory);
@@ -177,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);
@@ -262,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);
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/SearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index 5847520..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;