Added InvertedIndexSearchOperator and first integration test.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@786 123451ca-8445-de46-9d55-352943316053
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
index 26ac7dc..19eec77 100644
--- 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
@@ -60,9 +60,13 @@
 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;
@@ -129,6 +133,13 @@
     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 {
@@ -151,12 +162,15 @@
 
         loadPrimaryIndex();
         printPrimaryIndex();
-        loadInvertedIndex();
+        loadInvertedIndex();        
     }
 
     @Test
-    public void test() {
-        // TODO: perform inverted index searches.
+    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) {
@@ -242,9 +256,6 @@
     }
 
     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);
@@ -279,4 +290,44 @@
         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-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 4dc655e..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
@@ -56,7 +56,7 @@
             RecordDescriptor recDesc, IStorageManagerInterface storageManager,
             IFileSplitProvider btreeFileSplitProvider, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
             ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
-            ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
+            ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories,
             ITreeIndexOpHelperFactory btreeOpHelperFactory, IFileSplitProvider invListsFileSplitProvider,
             IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories) {
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 a250750..09f4c84 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
@@ -46,7 +46,7 @@
             IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
             IBinaryComparatorFactory[] invListComparatorFactories) {
         super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, treeIndexRegistryProvider,
-                btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeFillFactor,
+                btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories, 
                 btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider, invListsTypeTraits,
                 invListComparatorFactories);
         this.fieldPermutation = fieldPermutation;
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 7df754a..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
@@ -117,7 +117,11 @@
         } catch (PageAllocationException e) {
             throw new HyracksDataException(e);
         } finally {
-            btreeOpHelper.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/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..89d8f9a
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
@@ -0,0 +1,57 @@
+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.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.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, IOperatorEnvironment env,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        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/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index 3cfa1d0..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
@@ -141,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();
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);
+    }
+}