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.