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);
+ }
+}