Changed partitioning field in length-partitioned inverted indexes from integer to short.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_lsm_length_filter@2486 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/ShortSerializerDeserializer.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/ShortSerializerDeserializer.java
new file mode 100644
index 0000000..15384b3
--- /dev/null
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/ShortSerializerDeserializer.java
@@ -0,0 +1,58 @@
+/*
+ * 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.dataflow.common.data.marshalling;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ShortSerializerDeserializer implements ISerializerDeserializer<Short> {
+ private static final long serialVersionUID = 1L;
+
+ public static final ShortSerializerDeserializer INSTANCE = new ShortSerializerDeserializer();
+
+ private ShortSerializerDeserializer() {
+ }
+
+ @Override
+ public Short deserialize(DataInput in) throws HyracksDataException {
+ try {
+ return in.readShort();
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void serialize(Short instance, DataOutput out) throws HyracksDataException {
+ try {
+ out.writeShort(instance.intValue());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public static short getShort(byte[] bytes, int offset) {
+ return (short) (((bytes[offset] & 0xff) << 8) + ((bytes[offset + 1] & 0xff)));
+ }
+
+ public static void putShort(int val, byte[] bytes, int offset) {
+ bytes[offset] = (byte) ((val >>> 8) & 0xFF);
+ bytes[offset + 1] = (byte) ((val >>> 0) & 0xFF);
+ }
+}
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
index 00575f4..9dafa83 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/SerdeUtils.java
@@ -25,43 +25,45 @@
import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.data.std.primitive.LongPointable;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@SuppressWarnings("rawtypes")
public class SerdeUtils {
- public static class PayloadTypeTraits implements ITypeTraits {
- private static final long serialVersionUID = 1L;
- final int payloadSize;
-
- public PayloadTypeTraits(int payloadSize) {
- this.payloadSize = payloadSize;
- }
-
- @Override
- public boolean isFixedLength() {
- return true;
- }
+ public static class PayloadTypeTraits implements ITypeTraits {
+ private static final long serialVersionUID = 1L;
+ final int payloadSize;
- @Override
- public int getFixedLength() {
- return payloadSize;
- }
- }
-
- public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes) {
+ public PayloadTypeTraits(int payloadSize) {
+ this.payloadSize = payloadSize;
+ }
+
+ @Override
+ public boolean isFixedLength() {
+ return true;
+ }
+
+ @Override
+ public int getFixedLength() {
+ return payloadSize;
+ }
+ }
+
+ public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes) {
ITypeTraits[] typeTraits = new ITypeTraits[serdes.length];
for (int i = 0; i < serdes.length; i++) {
typeTraits[i] = serdeToTypeTrait(serdes[i]);
}
return typeTraits;
}
-
+
public static ITypeTraits[] serdesToTypeTraits(ISerializerDeserializer[] serdes, int payloadSize) {
ITypeTraits[] typeTraits = new ITypeTraits[serdes.length + 1];
for (int i = 0; i < serdes.length; i++) {
@@ -72,6 +74,9 @@
}
public static ITypeTraits serdeToTypeTrait(ISerializerDeserializer serde) {
+ if (serde instanceof ShortSerializerDeserializer) {
+ return ShortPointable.TYPE_TRAITS;
+ }
if (serde instanceof IntegerSerializerDeserializer) {
return IntegerPointable.TYPE_TRAITS;
}
@@ -112,6 +117,9 @@
}
public static IBinaryComparatorFactory serdeToComparatorFactory(ISerializerDeserializer serde) {
+ if (serde instanceof ShortSerializerDeserializer) {
+ return PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+ }
if (serde instanceof IntegerSerializerDeserializer) {
return PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java
index afe082d..0d0d936 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IInvertedIndexSearchModifier.java
@@ -15,13 +15,12 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api;
-
public interface IInvertedIndexSearchModifier {
public int getOccurrenceThreshold(int numQueryTokens);
public int getNumPrefixLists(int occurrenceThreshold, int numInvLists);
-
- public int getNumTokensLowerBound(int numQueryTokens);
-
- public int getNumTokensUpperBound(int numQueryTokens);
+
+ public short getNumTokensLowerBound(short numQueryTokens);
+
+ public short getNumTokensUpperBound(short numQueryTokens);
}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java
index 98cc476..7db972c 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/api/IPartitionedInvertedIndex.java
@@ -22,6 +22,6 @@
public interface IPartitionedInvertedIndex {
public void openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
- int numTokensLowerBound, int numTokensUpperBound, InvertedListPartitions invListPartitions)
+ short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions)
throws HyracksDataException, IndexException;
}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index dc5610f..53def29 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -40,8 +40,8 @@
public class PartitionedInMemoryInvertedIndex extends InMemoryInvertedIndex implements IPartitionedInvertedIndex {
protected final ReentrantReadWriteLock partitionIndexLock = new ReentrantReadWriteLock(true);
- protected int minPartitionIndex = Integer.MAX_VALUE;
- protected int maxPartitionIndex = Integer.MIN_VALUE;
+ protected short minPartitionIndex = Short.MAX_VALUE;
+ protected short maxPartitionIndex = Short.MIN_VALUE;
public PartitionedInMemoryInvertedIndex(IBufferCache memBufferCache, IFreePageManager memFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
@@ -63,11 +63,11 @@
@Override
public void clear() throws HyracksDataException {
super.clear();
- minPartitionIndex = Integer.MAX_VALUE;
- maxPartitionIndex = Integer.MIN_VALUE;
+ minPartitionIndex = Short.MAX_VALUE;
+ maxPartitionIndex = Short.MIN_VALUE;
}
- public void updatePartitionIndexes(int numTokens) {
+ public void updatePartitionIndexes(short numTokens) {
partitionIndexLock.writeLock().lock();
if (numTokens < minPartitionIndex) {
minPartitionIndex = numTokens;
@@ -87,26 +87,26 @@
@Override
public void openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
- int numTokensLowerBound, int numTokensUpperBound, InvertedListPartitions invListPartitions)
+ short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions)
throws HyracksDataException, IndexException {
- int minPartitionIndex;
- int maxPartitionIndex;
+ short minPartitionIndex;
+ short maxPartitionIndex;
partitionIndexLock.readLock().lock();
minPartitionIndex = this.minPartitionIndex;
maxPartitionIndex = this.maxPartitionIndex;
partitionIndexLock.readLock().unlock();
- if (minPartitionIndex == Integer.MAX_VALUE || maxPartitionIndex == Integer.MIN_VALUE) {
+ if (minPartitionIndex == Short.MAX_VALUE || maxPartitionIndex == Short.MIN_VALUE) {
// Index must be empty.
return;
}
- int partitionStartIndex = minPartitionIndex;
- int partitionEndIndex = maxPartitionIndex;
+ short partitionStartIndex = minPartitionIndex;
+ short partitionEndIndex = maxPartitionIndex;
if (numTokensLowerBound >= 0) {
- partitionStartIndex = Math.max(minPartitionIndex, numTokensLowerBound);
+ partitionStartIndex = (short) Math.max(minPartitionIndex, numTokensLowerBound);
}
if (numTokensUpperBound >= 0) {
- partitionEndIndex = Math.min(maxPartitionIndex, numTokensUpperBound);
+ partitionEndIndex = (short) Math.min(maxPartitionIndex, numTokensUpperBound);
}
PartitionedTOccurrenceSearcher partSearcher = (PartitionedTOccurrenceSearcher) searcher;
@@ -119,7 +119,7 @@
// Go through all possibly partitions and see if the token matches.
// TODO: This procedure could be made more efficient by determining the next partition to search
// using the last existing partition and re-searching the BTree with an open interval as low key.
- for (int i = partitionStartIndex; i <= partitionEndIndex; i++) {
+ for (short i = partitionStartIndex; i <= partitionEndIndex; i++) {
partSearcher.setNumTokensBoundsInSearchKeys(i, i);
InMemoryInvertedListCursor inMemListCursor = (InMemoryInvertedListCursor) partSearcher
.getCachedInvertedListCursor();
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index 84228ac..5b2ec60 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -20,7 +20,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -62,7 +62,7 @@
@Override
public void openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
- int numTokensLowerBound, int numTokensUpperBound, InvertedListPartitions invListPartitions)
+ short numTokensLowerBound, short numTokensUpperBound, InvertedListPartitions invListPartitions)
throws HyracksDataException, IndexException {
PartitionedTOccurrenceSearcher partSearcher = (PartitionedTOccurrenceSearcher) searcher;
OnDiskInvertedIndexOpContext ctx = (OnDiskInvertedIndexOpContext) ictx;
@@ -90,7 +90,7 @@
while (ctx.btreeCursor.hasNext()) {
ctx.btreeCursor.next();
ITupleReference btreeTuple = ctx.btreeCursor.getTuple();
- int numTokens = IntegerSerializerDeserializer.getInt(
+ short numTokens = ShortSerializerDeserializer.getShort(
btreeTuple.getFieldData(PARTITIONING_NUM_TOKENS_FIELD),
btreeTuple.getFieldStart(PARTITIONING_NUM_TOKENS_FIELD));
IInvertedListCursor invListCursor = partSearcher.getCachedInvertedListCursor();
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java
index 55c5f30..318f1e1 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/ConjunctiveSearchModifier.java
@@ -35,12 +35,12 @@
}
@Override
- public int getNumTokensLowerBound(int numQueryTokens) {
+ public short getNumTokensLowerBound(short numQueryTokens) {
return -1;
}
@Override
- public int getNumTokensUpperBound(int numQueryTokens) {
+ public short getNumTokensUpperBound(short numQueryTokens) {
return -1;
}
}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java
index 26e11ba..9c06f4d 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/EditDistanceSearchModifier.java
@@ -38,15 +38,15 @@
}
@Override
- public int getNumTokensLowerBound(int numQueryTokens) {
- return numQueryTokens - edThresh;
+ public short getNumTokensLowerBound(short numQueryTokens) {
+ return (short) (numQueryTokens - edThresh);
}
@Override
- public int getNumTokensUpperBound(int numQueryTokens) {
- return numQueryTokens + edThresh;
+ public short getNumTokensUpperBound(short numQueryTokens) {
+ return (short) (numQueryTokens + edThresh);
}
-
+
public int getGramLength() {
return gramLength;
}
@@ -62,7 +62,7 @@
public void setEdThresh(int edThresh) {
this.edThresh = edThresh;
}
-
+
@Override
public String toString() {
return "Edit Distance Search Modifier, GramLen: " + gramLength + ", Threshold: " + edThresh;
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java
index 99e61bd..2f720fb 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/InvertedListPartitions.java
@@ -40,7 +40,7 @@
}
@SuppressWarnings("unchecked")
- public void reset(int numTokensLowerBound, int numTokensUpperBound) {
+ public void reset(short numTokensLowerBound, short numTokensUpperBound) {
if (partitions == null) {
int initialSize;
if (numTokensUpperBound < 0) {
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java
index b005905..ede6041 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/JaccardSearchModifier.java
@@ -39,15 +39,15 @@
}
@Override
- public int getNumTokensLowerBound(int numQueryTokens) {
- return (int) Math.floor(numQueryTokens * jaccThresh);
+ public short getNumTokensLowerBound(short numQueryTokens) {
+ return (short) Math.floor(numQueryTokens * jaccThresh);
}
@Override
- public int getNumTokensUpperBound(int numQueryTokens) {
- return (int) Math.ceil(numQueryTokens / jaccThresh);
+ public short getNumTokensUpperBound(short numQueryTokens) {
+ return (short) Math.ceil(numQueryTokens / jaccThresh);
}
-
+
public float getJaccThresh() {
return jaccThresh;
}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
index 4f33339..6b482ba 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -23,7 +23,7 @@
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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
@@ -54,7 +54,7 @@
try {
lowerBoundTupleBuilder.reset();
// Write dummy value.
- lowerBoundTupleBuilder.getDataOutput().writeInt(Integer.MIN_VALUE);
+ lowerBoundTupleBuilder.getDataOutput().writeShort(Short.MIN_VALUE);
lowerBoundTupleBuilder.addFieldEndOffset();
lowerBoundTuple.reset(lowerBoundTupleBuilder.getFieldEndOffsets(), lowerBoundTupleBuilder.getByteArray());
// Only needed for setting the number of fields in searchKey.
@@ -65,7 +65,7 @@
upperBoundTupleBuilder.reset();
// Write dummy value.
- upperBoundTupleBuilder.getDataOutput().writeInt(Integer.MAX_VALUE);
+ upperBoundTupleBuilder.getDataOutput().writeShort(Short.MAX_VALUE);
upperBoundTupleBuilder.addFieldEndOffset();
upperBoundTuple.reset(upperBoundTupleBuilder.getFieldEndOffsets(), upperBoundTupleBuilder.getByteArray());
// Only needed for setting the number of fields in searchKey.
@@ -81,11 +81,11 @@
public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred,
IIndexOperationContext ictx) throws HyracksDataException, IndexException {
tokenizeQuery(searchPred);
- int numQueryTokens = queryTokenAccessor.getTupleCount();
+ short numQueryTokens = (short) queryTokenAccessor.getTupleCount();
IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
- int numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
- int numTokensUpperBound = searchModifier.getNumTokensUpperBound(numQueryTokens);
+ short numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
+ short numTokensUpperBound = searchModifier.getNumTokensUpperBound(numQueryTokens);
IPartitionedInvertedIndex partInvIndex = (IPartitionedInvertedIndex) invIndex;
invListCursorCache.reset();
@@ -123,10 +123,10 @@
resultCursor.open(null, searchPred);
}
- public void setNumTokensBoundsInSearchKeys(int numTokensLowerBound, int numTokensUpperBound) {
- IntegerSerializerDeserializer.putInt(numTokensLowerBound, lowerBoundTuple.getFieldData(0),
+ public void setNumTokensBoundsInSearchKeys(short numTokensLowerBound, short numTokensUpperBound) {
+ ShortSerializerDeserializer.putShort(numTokensLowerBound, lowerBoundTuple.getFieldData(0),
lowerBoundTuple.getFieldStart(0));
- IntegerSerializerDeserializer.putInt(numTokensUpperBound, upperBoundTuple.getFieldData(0),
+ ShortSerializerDeserializer.putShort(numTokensUpperBound, upperBoundTuple.getFieldData(0),
upperBoundTuple.getFieldStart(0));
}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
index e0477b2..a6a4bc1 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
@@ -25,7 +25,7 @@
// TODO: We can possibly avoid copying the data into a new tuple here.
public class PartitionedInvertedIndexTokenizingTupleIterator extends InvertedIndexTokenizingTupleIterator {
- protected int numTokens = 0;
+ protected short numTokens = 0;
public PartitionedInvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount,
IBinaryTokenizer tokenizer) {
@@ -52,7 +52,7 @@
token.serializeToken(tupleBuilder.getDataOutput());
tupleBuilder.addFieldEndOffset();
// Add field with number of tokens.
- tupleBuilder.getDataOutput().writeInt(numTokens);
+ tupleBuilder.getDataOutput().writeShort(numTokens);
tupleBuilder.addFieldEndOffset();
} catch (IOException e) {
throw new HyracksDataException(e);
@@ -65,8 +65,8 @@
// Reset tuple reference for insert operation.
tupleReference.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
}
-
- public int getNumTokens() {
+
+ public short getNumTokens() {
return numTokens;
}
}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index 38896fb..3bd8129 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -40,6 +40,7 @@
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.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.OrderedIndexTestUtils;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
@@ -115,7 +116,7 @@
case PARTITIONED_LSM: {
// Such indexes also include the set-size for partitioning.
fieldSerdes = new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+ ShortSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
break;
}
default: {
@@ -141,7 +142,7 @@
case PARTITIONED_LSM: {
// Such indexes also include the set-size for partitioning.
fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+ ShortSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
break;
}
default: {
@@ -419,12 +420,12 @@
tokenizer.next();
numQueryTokens++;
}
- int numTokensLowerBound = -1;
- int numTokensUpperBound = -1;
+ short numTokensLowerBound = -1;
+ short numTokensUpperBound = -1;
int invListElementField = 1;
if (isPartitioned) {
- numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
- numTokensUpperBound = searchModifier.getNumTokensUpperBound(numQueryTokens);
+ numTokensLowerBound = searchModifier.getNumTokensLowerBound((short) numQueryTokens);
+ numTokensUpperBound = searchModifier.getNumTokensUpperBound((short) numQueryTokens);
invListElementField = 2;
}
int occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
@@ -448,7 +449,7 @@
// Index is length partitioned, and search modifier supports length filtering.
lowKey = new CheckTuple(2, 2);
lowKey.appendField(tokenObj);
- lowKey.appendField(Integer.valueOf(numTokensLowerBound));
+ lowKey.appendField(Short.valueOf(numTokensLowerBound));
}
CheckTuple highKey;
if (numTokensUpperBound < 0) {
@@ -459,7 +460,7 @@
// Index is length partitioned, and search modifier supports length filtering.
highKey = new CheckTuple(2, 2);
highKey.appendField(tokenObj);
- highKey.appendField(Integer.valueOf(numTokensUpperBound));
+ highKey.appendField(Short.valueOf(numTokensUpperBound));
}
// Get view over check tuples containing inverted-list corresponding to token.