Added prefix-search capability for sorted sets of CheckTuples to enable comparing inverted indexes against a baseline inverted index.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_inverted_index_updates_new@1819 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 05c3cbe..4699a1a 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory;
import java.io.File;
-import java.io.IOException;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -39,8 +38,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
public class InMemoryInvertedIndex implements IInvertedIndex {
@@ -51,20 +49,20 @@
private final IBinaryComparatorFactory[] tokenCmpFactories;
private final ITypeTraits[] invListTypeTraits;
private final IBinaryComparatorFactory[] invListCmpFactories;
- private final IBinaryTokenizer tokenizer;
+ private final IBinaryTokenizerFactory tokenizerFactory;
private final ITypeTraits[] btreeTypeTraits;
private final IBinaryComparatorFactory[] btreeCmpFactories;
public InMemoryInvertedIndex(IBufferCache memBufferCache, IFreePageManager memFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
- ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizer tokenizer)
- throws BTreeException {
+ ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+ IBinaryTokenizerFactory tokenizerFactory) throws BTreeException {
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
- this.tokenizer = tokenizer;
+ this.tokenizerFactory = tokenizerFactory;
// BTree tuples: <tokens, inverted-list elements>.
int numBTreeFields = tokenTypeTraits.length + invListTypeTraits.length;
btreeTypeTraits = new ITypeTraits[numBTreeFields];
@@ -115,29 +113,12 @@
public boolean insert(ITupleReference tuple, BTreeAccessor btreeAccessor, IIndexOpContext ictx)
throws HyracksDataException, IndexException {
InMemoryInvertedIndexOpContext ctx = (InMemoryInvertedIndexOpContext) ictx;
- // TODO: We can possibly avoid copying the data into a new tuple here.
- tokenizer.reset(tuple.getFieldData(0), tuple.getFieldStart(0), tuple.getFieldLength(0));
- while (tokenizer.hasNext()) {
- tokenizer.next();
- IToken token = tokenizer.getToken();
- ctx.btreeTupleBuilder.reset();
- // Add token field.
+ ctx.insertTupleIter.reset(tuple);
+ while (ctx.insertTupleIter.hasNext()) {
+ ctx.insertTupleIter.next();
+ ITupleReference insertTuple = ctx.insertTupleIter.getTuple();
try {
- token.serializeToken(ctx.btreeTupleBuilder.getDataOutput());
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- ctx.btreeTupleBuilder.addFieldEndOffset();
- // Add inverted-list element fields.
- for (int i = 0; i < invListTypeTraits.length; i++) {
- ctx.btreeTupleBuilder.addField(tuple.getFieldData(i + 1), tuple.getFieldStart(i + 1),
- tuple.getFieldLength(i + 1));
- }
- // Reset tuple reference for insert operation.
- ctx.btreeTupleReference.reset(ctx.btreeTupleBuilder.getFieldEndOffsets(),
- ctx.btreeTupleBuilder.getByteArray());
- try {
- btreeAccessor.insert(ctx.btreeTupleReference);
+ btreeAccessor.insert(insertTuple);
} catch (BTreeDuplicateKeyException e) {
// This exception may be caused by duplicate tokens in the same insert "document".
// We ignore such duplicate tokens in all inverted-index implementations, hence
@@ -159,8 +140,8 @@
}
@Override
- public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference tupleReference, IIndexOpContext ictx)
- throws HyracksDataException, IndexException {
+ public void openInvertedListCursor(IInvertedListCursor listCursor, ITupleReference tupleReference,
+ IIndexOpContext ictx) throws HyracksDataException, IndexException {
InMemoryInvertedIndexOpContext ctx = (InMemoryInvertedIndexOpContext) ictx;
InMemoryInvertedListCursor inMemListCursor = (InMemoryInvertedListCursor) listCursor;
inMemListCursor.prepare(ctx.btreeAccessor, ctx.btreePred, ctx.tokenFieldsCmp, ctx.btreeCmp);
@@ -170,13 +151,13 @@
@Override
public IIndexAccessor createAccessor(IModificationOperationCallback modificationCallback,
ISearchOperationCallback searchCallback) {
- return new InMemoryInvertedIndexAccessor(this, new InMemoryInvertedIndexOpContext(
- btree, tokenCmpFactories), tokenizer);
+ return new InMemoryInvertedIndexAccessor(this, new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories,
+ tokenizerFactory));
}
@Override
public IBufferCache getBufferCache() {
- return null;
+ return btree.getBufferCache();
}
@Override
@@ -197,7 +178,7 @@
public ITypeTraits[] getInvListTypeTraits() {
return invListTypeTraits;
}
-
+
@Override
public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput) throws IndexException {
throw new UnsupportedOperationException("Bulk load not supported by in-memory inverted index.");
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
index e69716e..628c726 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java
@@ -27,33 +27,31 @@
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex.DefaultHyracksCommonContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexSearchCursor;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
public class InMemoryInvertedIndexAccessor implements IIndexAccessor {
// TODO: This ctx needs to go away.
protected final IHyracksCommonContext hyracksCtx = new DefaultHyracksCommonContext();
protected final IInvertedIndexSearcher searcher;
protected IIndexOpContext opCtx;
- protected InMemoryInvertedIndex memoryBtreeInvertedIndex;
+ protected InMemoryInvertedIndex inMemInvIx;
protected BTreeAccessor btreeAccessor;
- public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex memoryBtreeInvertedIndex,
- IIndexOpContext opCtx, IBinaryTokenizer tokenizer) {
+ public InMemoryInvertedIndexAccessor(InMemoryInvertedIndex inMemInvIx, IIndexOpContext opCtx) {
this.opCtx = opCtx;
- this.memoryBtreeInvertedIndex = memoryBtreeInvertedIndex;
- this.searcher = new TOccurrenceSearcher(hyracksCtx, memoryBtreeInvertedIndex);
+ this.inMemInvIx = inMemInvIx;
+ this.searcher = new TOccurrenceSearcher(hyracksCtx, inMemInvIx);
// TODO: Ignore opcallbacks for now.
- this.btreeAccessor = (BTreeAccessor) memoryBtreeInvertedIndex.getBTree().createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ this.btreeAccessor = (BTreeAccessor) inMemInvIx.getBTree().createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
}
public void insert(ITupleReference tuple) throws HyracksDataException, IndexException {
opCtx.reset(IndexOp.INSERT);
- memoryBtreeInvertedIndex.insert(tuple, btreeAccessor, opCtx);
+ inMemInvIx.insert(tuple, btreeAccessor, opCtx);
}
@Override
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
index 0dd9774..5009c73 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
@@ -16,8 +16,6 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
@@ -25,6 +23,9 @@
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexInsertTupleIterator;
public class InMemoryInvertedIndexOpContext implements IIndexOpContext {
public IndexOp op;
@@ -38,20 +39,23 @@
public MultiComparator tokenFieldsCmp;
// To generate in-memory BTree tuples for insertions.
- public ArrayTupleBuilder btreeTupleBuilder;
- public ArrayTupleReference btreeTupleReference;
+ private final IBinaryTokenizerFactory tokenizerFactory;
+ public InvertedIndexInsertTupleIterator insertTupleIter;
- public InMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories) {
+ public InMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
+ IBinaryTokenizerFactory tokenizerFactory) {
this.btree = btree;
this.tokenCmpFactories = tokenCmpFactories;
+ this.tokenizerFactory = tokenizerFactory;
}
@Override
public void reset(IndexOp newOp) {
switch (newOp) {
case INSERT: {
- btreeTupleBuilder = new ArrayTupleBuilder(btree.getFieldCount());
- btreeTupleReference = new ArrayTupleReference();
+ IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+ insertTupleIter = new InvertedIndexInsertTupleIterator(tokenCmpFactories.length, btree.getFieldCount()
+ - tokenCmpFactories.length, tokenizer);
break;
}
case SEARCH: {
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexInsertTupleIterator.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexInsertTupleIterator.java
new file mode 100644
index 0000000..44ef3c1
--- /dev/null
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexInsertTupleIterator.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2012 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.lsm.invertedindex.util;
+
+import java.io.IOException;
+
+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.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+// TODO: We can possibly avoid copying the data into a new tuple here.
+public class InvertedIndexInsertTupleIterator {
+ // Field that is expected to be tokenized.
+ private final int DOC_FIELD_INDEX = 0;
+
+ private final int invListFieldCount;
+ private final ArrayTupleBuilder tupleBuilder;
+ private final ArrayTupleReference tupleReference;
+ private final IBinaryTokenizer tokenizer;
+ private ITupleReference inputTuple;
+
+ public InvertedIndexInsertTupleIterator(int tokensFieldCount, int invListFieldCount, IBinaryTokenizer tokenizer) {
+ this.invListFieldCount = invListFieldCount;
+ this.tupleBuilder = new ArrayTupleBuilder(tokensFieldCount + invListFieldCount);
+ this.tupleReference = new ArrayTupleReference();
+ this.tokenizer = tokenizer;
+ }
+
+ public void reset(ITupleReference inputTuple) {
+ this.inputTuple = inputTuple;
+ tokenizer.reset(inputTuple.getFieldData(DOC_FIELD_INDEX), inputTuple.getFieldStart(DOC_FIELD_INDEX),
+ inputTuple.getFieldLength(DOC_FIELD_INDEX));
+ }
+
+ public boolean hasNext() {
+ return tokenizer.hasNext();
+ }
+
+ public void next() throws HyracksDataException {
+ tokenizer.next();
+ IToken token = tokenizer.getToken();
+ tupleBuilder.reset();
+ // Add token field.
+ try {
+ token.serializeToken(tupleBuilder.getDataOutput());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ tupleBuilder.addFieldEndOffset();
+ // Add inverted-list element fields.
+ for (int i = 0; i < invListFieldCount; i++) {
+ tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
+ inputTuple.getFieldLength(i + 1));
+ }
+ // Reset tuple reference for insert operation.
+ tupleReference.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ }
+
+ public ITupleReference getTuple() {
+ return tupleReference;
+ }
+}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index 73c5330..dceb16a 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeElementInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -36,9 +36,9 @@
public static InMemoryInvertedIndex createInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
IFreePageManager memFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizer tokenizer) throws BTreeException {
+ IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory) throws BTreeException {
return new InMemoryInvertedIndex(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizer);
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
}
public static OnDiskInvertedIndex createOnDiskInvertedIndex(IBufferCache bufferCache,
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index 605dbd7..5aaa7d9 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -7,8 +7,8 @@
import java.io.DataInputStream;
import java.util.Collection;
import java.util.Iterator;
-import java.util.NavigableSet;
import java.util.Random;
+import java.util.SortedSet;
import java.util.TreeSet;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -51,35 +51,27 @@
}
@SuppressWarnings("unchecked")
- // Create a new TreeSet containing the elements satisfying the prefix
- // search.
+ // Create a new TreeSet containing the elements satisfying the prefix search.
// Implementing prefix search by changing compareTo() in CheckTuple does not
// work.
- public static TreeSet<CheckTuple> getPrefixExpectedSubset(TreeSet<CheckTuple> checkTuples, CheckTuple lowKey,
+ public static SortedSet<CheckTuple> getPrefixExpectedSubset(TreeSet<CheckTuple> checkTuples, CheckTuple lowKey,
CheckTuple highKey) {
- TreeSet<CheckTuple> expectedSubset = new TreeSet<CheckTuple>();
- Iterator<CheckTuple> iter = checkTuples.iterator();
- while (iter.hasNext()) {
- CheckTuple t = iter.next();
- boolean geLowKey = true;
- boolean leHighKey = true;
- for (int i = 0; i < lowKey.getNumKeys(); i++) {
- if (t.getField(i).compareTo(lowKey.getField(i)) < 0) {
- geLowKey = false;
- break;
- }
- }
- for (int i = 0; i < highKey.getNumKeys(); i++) {
- if (t.getField(i).compareTo(highKey.getField(i)) > 0) {
- leHighKey = false;
- break;
- }
- }
- if (geLowKey && leHighKey) {
- expectedSubset.add(t);
- }
+ lowKey.setIsHighKey(false);
+ highKey.setIsHighKey(true);
+ CheckTuple low = checkTuples.ceiling(lowKey);
+ CheckTuple high = checkTuples.floor(highKey);
+ if (low == null) {
+ // Must be empty.
+ return new TreeSet<CheckTuple>();
}
- return expectedSubset;
+ if (high == null) {
+ throw new IllegalStateException("Upper bound is null.");
+ }
+ if (high.compareTo(low) < 0) {
+ // Must be empty.
+ return new TreeSet<CheckTuple>();
+ }
+ return checkTuples.subSet(low, true, high, true);
}
@SuppressWarnings("unchecked")
@@ -99,7 +91,7 @@
CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, ctx.getFieldSerdes(), lowKeyCmp.getKeyFieldCount());
CheckTuple highKeyCheck = createCheckTupleFromTuple(highKey, ctx.getFieldSerdes(),
highKeyCmp.getKeyFieldCount());
- NavigableSet<CheckTuple> expectedSubset = null;
+ SortedSet<CheckTuple> expectedSubset = null;
if (lowKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()
|| highKeyCmp.getKeyFieldCount() < ctx.getKeyFieldCount()) {
// Searching on a key prefix (low key or high key or both).
@@ -207,10 +199,10 @@
}
}
}
-
+
public void upsertStringTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
- OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
- int fieldCount = ctx.getFieldCount();
+ OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
+ int fieldCount = ctx.getFieldCount();
int numKeyFields = ctx.getKeyFieldCount();
String[] fieldValues = new String[fieldCount];
for (int i = 0; i < numTuples; i++) {
@@ -264,7 +256,7 @@
public void upsertIntTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
- int fieldCount = ctx.getFieldCount();
+ int fieldCount = ctx.getFieldCount();
int numKeyFields = ctx.getKeyFieldCount();
int[] fieldValues = new int[ctx.getFieldCount()];
// Scale range of values according to number of keys.
@@ -286,7 +278,7 @@
ctx.upsertCheckTuple(createIntCheckTuple(fieldValues, ctx.getKeyFieldCount()), ctx.getCheckTuples());
}
}
-
+
@SuppressWarnings("unchecked")
public void updateTuples(IIndexTestContext ictx, int numTuples, Random rnd) throws Exception {
OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx;
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
index 9e31f71..b7037d6 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/CheckTuple.java
@@ -20,11 +20,13 @@
protected final int numKeys;
protected final Comparable[] fields;
protected int pos;
+ protected boolean isHighKey;
public CheckTuple(int numFields, int numKeys) {
this.numKeys = numKeys;
this.fields = new Comparable[numFields];
pos = 0;
+ isHighKey = false;
}
public void appendField(T e) {
@@ -34,12 +36,22 @@
@Override
public int compareTo(T o) {
CheckTuple<T> other = (CheckTuple<T>) o;
- for (int i = 0; i < numKeys; i++) {
+ int cmpFieldCount = Math.min(other.getNumKeys(), numKeys);
+ for (int i = 0; i < cmpFieldCount; i++) {
int cmp = fields[i].compareTo(other.getField(i));
if (cmp != 0) {
return cmp;
}
}
+ if (other.getNumKeys() == numKeys) {
+ return 0;
+ }
+ if (other.getNumKeys() < numKeys) {
+ return (other.isHighKey) ? -1 : 1;
+ }
+ if (other.getNumKeys() > numKeys) {
+ return (isHighKey) ? 1 : -1;
+ }
return 0;
}
@@ -53,7 +65,6 @@
@Override
public int hashCode() {
- //int hash = 37 * numKeys + fields.length;
int hash = 0;
for (int i = 0; i < numKeys; i++) {
hash = 37 * hash + fields[i].hashCode();
@@ -61,6 +72,10 @@
return hash;
}
+ public void setIsHighKey(boolean isHighKey) {
+ this.isHighKey = isHighKey;
+ }
+
public T getField(int idx) {
return (T) fields[idx];
}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/AbstractInvertedIndexTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/AbstractInvertedIndexTest.java
index 351b772..cba8959 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/AbstractInvertedIndexTest.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/AbstractInvertedIndexTest.java
@@ -74,6 +74,7 @@
protected int numDocuments = 1000;
protected int docMinWords = 2;
protected int docMaxWords = 10;
+ protected List<String> documents = new ArrayList<String>();
protected Map<String, SortedSet<Integer>> baselineInvertedIndex = new HashMap<String, SortedSet<Integer>>();
// Generate random data is false by default (generate predefined data is true!)
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTestContext.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTestContext.java
index 3349d3a..134c421 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTestContext.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTestContext.java
@@ -28,7 +28,7 @@
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.InvertedIndexException;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -71,21 +71,10 @@
return allCmpFactories;
}
- @Override
- public void insertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
- // TODO: Tokenize and insert <token,key> pairs.
- checkTuples.add(checkTuple);
- }
-
- @Override
- public void deleteCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
- // TODO: Tokenize and insert <token,key> pairs.
- checkTuples.remove(checkTuple);
- }
-
public static InvertedIndexTestContext create(IBufferCache bufferCache, IFreePageManager freePageManager,
IFileMapProvider fileMapProvider, FileReference invListsFile, ISerializerDeserializer[] fieldSerdes,
- int tokenFieldCount, IBinaryTokenizer tokenizer, InvertedIndexType invIndexType) throws Exception {
+ int tokenFieldCount, IBinaryTokenizerFactory tokenizerFactory, InvertedIndexType invIndexType)
+ throws Exception {
ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IBinaryComparatorFactory[] allCmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes,
fieldSerdes.length);
@@ -109,7 +98,7 @@
switch (invIndexType) {
case INMEMORY: {
invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(bufferCache, freePageManager,
- invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizer);
+ invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
break;
}
case ONDISK: {
@@ -124,7 +113,7 @@
InvertedIndexTestContext testCtx = new InvertedIndexTestContext(fieldSerdes, invIndex);
return testCtx;
}
-
+
@Override
public void upsertCheckTuple(CheckTuple checkTuple, Collection<CheckTuple> checkTuples) {
throw new UnsupportedOperationException("Upsert not supported by inverted index.");