Implemented length-partitioned LSM inverted index. Still some cleanup needed.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_lsm_length_filter@2467 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 2ee1fe1..1e79d5c 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -96,33 +96,33 @@
         }
     }
 
-    private final ILSMHarness lsmHarness;
+    protected final ILSMHarness lsmHarness;
 
     // In-memory components.
-    private final LSMInvertedIndexComponent memComponent;
-    private final IInMemoryFreePageManager memFreePageManager;
-    private final IBinaryTokenizerFactory tokenizerFactory;
+    protected final LSMInvertedIndexComponent memComponent;
+    protected final IInMemoryFreePageManager memFreePageManager;
+    protected final IBinaryTokenizerFactory tokenizerFactory;
 
     // On-disk components.
-    private final ILSMIndexFileManager fileManager;
+    protected final ILSMIndexFileManager fileManager;
     // For creating inverted indexes in flush and merge.
-    private final OnDiskInvertedIndexFactory diskInvIndexFactory;
+    protected final OnDiskInvertedIndexFactory diskInvIndexFactory;
     // For creating deleted-keys BTrees in flush and merge.
-    private final BTreeFactory deletedKeysBTreeFactory;
-    private final IBufferCache diskBufferCache;
+    protected final BTreeFactory deletedKeysBTreeFactory;
+    protected final IBufferCache diskBufferCache;
     // List of LSMInvertedIndexComponent instances. Using Object for better sharing via
     // ILSMIndex + LSMHarness.
-    private final LinkedList<Object> diskComponents;
+    protected final LinkedList<Object> diskComponents;
     // Helps to guarantees physical consistency of LSM components.
-    private final ILSMComponentFinalizer componentFinalizer;
+    protected final ILSMComponentFinalizer componentFinalizer;
 
     // Type traits and comparators for tokens and inverted-list elements.
-    private final ITypeTraits[] invListTypeTraits;
-    private final IBinaryComparatorFactory[] invListCmpFactories;
-    private final ITypeTraits[] tokenTypeTraits;
-    private final IBinaryComparatorFactory[] tokenCmpFactories;
+    protected final ITypeTraits[] invListTypeTraits;
+    protected final IBinaryComparatorFactory[] invListCmpFactories;
+    protected final ITypeTraits[] tokenTypeTraits;
+    protected final IBinaryComparatorFactory[] tokenCmpFactories;
 
-    private boolean isActivated;
+    protected boolean isActivated;
 
     public LSMInvertedIndex(IInMemoryBufferCache memBufferCache, IInMemoryFreePageManager memFreePageManager,
             OnDiskInvertedIndexFactory diskInvIndexFactory, BTreeFactory deletedKeysBTreeFactory,
@@ -131,13 +131,6 @@
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             ILSMFlushController flushController, ILSMMergePolicy mergePolicy, ILSMOperationTrackerFactory opTrackerFactory,
             ILSMIOOperationScheduler ioScheduler) throws IndexException {
-        InMemoryInvertedIndex memInvIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(memBufferCache,
-                memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                tokenizerFactory);
-        BTree deleteKeysBTree = BTreeUtils.createBTree(memBufferCache, memFreePageManager,
-                ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), invListTypeTraits, invListCmpFactories,
-                BTreeLeafFrameType.REGULAR_NSM, new FileReference(new File("membtree")));
-        memComponent = new LSMInvertedIndexComponent(memInvIndex, deleteKeysBTree);
         this.memFreePageManager = memFreePageManager;
         this.tokenizerFactory = tokenizerFactory;
         this.fileManager = fileManager;
@@ -153,6 +146,18 @@
         this.componentFinalizer = new LSMInvertedIndexComponentFinalizer(diskFileMapProvider);
         diskComponents = new LinkedList<Object>();
         isActivated = false;
+        // Create in-memory component.
+        InMemoryInvertedIndex memInvIndex = createInMemoryInvertedIndex(memBufferCache);
+        BTree deleteKeysBTree = BTreeUtils.createBTree(memBufferCache, memFreePageManager,
+                ((InMemoryBufferCache) memBufferCache).getFileMapProvider(), invListTypeTraits, invListCmpFactories,
+                BTreeLeafFrameType.REGULAR_NSM, new FileReference(new File("membtree")));
+        memComponent = new LSMInvertedIndexComponent(memInvIndex, deleteKeysBTree);
+    }
+    
+    protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
+            throws IndexException {
+        return InvertedIndexUtils.createInMemoryBTreeInvertedindex(memBufferCache, memFreePageManager,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
     }
 
     @Override
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
index b3dee89..625ecd4 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexOpContext.java
@@ -28,6 +28,8 @@
 
 public class LSMInvertedIndexOpContext implements ILSMIndexOperationContext {
 
+    private static final int NUM_DOCUMENT_FIELDS = 1;
+    
     private IndexOperation op;
     private final IInvertedIndex memInvIndex;
     private final IIndex memDeletedKeysBTree;
@@ -68,11 +70,10 @@
                     deletedKeysBTreeAccessor = memDeletedKeysBTree.createAccessor(NoOpOperationCallback.INSTANCE,
                             NoOpOperationCallback.INSTANCE);
                     // Project away the document fields, leaving only the key fields.
-                    int numTokenFields = memInvIndex.getTokenTypeTraits().length;
                     int numKeyFields = memInvIndex.getInvListTypeTraits().length;
-                    int[] keyFieldPermutation = new int[memInvIndex.getInvListTypeTraits().length];
+                    int[] keyFieldPermutation = new int[numKeyFields];
                     for (int i = 0; i < numKeyFields; i++) {
-                        keyFieldPermutation[i] = numTokenFields + i;
+                        keyFieldPermutation[i] = NUM_DOCUMENT_FIELDS + i;
                     }
                     keysOnlyTuple = new PermutingTupleReference(keyFieldPermutation);
                 }
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
new file mode 100644
index 0000000..d963dbc
--- /dev/null
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -0,0 +1,56 @@
+/*
+ * 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.impls;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BTreeFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
+
+    public PartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, OnDiskInvertedIndexFactory diskInvIndexFactory,
+            BTreeFactory deletedKeysBTreeFactory, ILSMIndexFileManager fileManager,
+            IFileMapProvider diskFileMapProvider, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+            ILSMFlushController flushController, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler) throws IndexException {
+        super(memBufferCache, memFreePageManager, diskInvIndexFactory, deletedKeysBTreeFactory, fileManager,
+                diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
+                tokenizerFactory, flushController, mergePolicy, opTrackerFactory, ioScheduler);
+    }
+
+    protected InMemoryInvertedIndex createInMemoryInvertedIndex(IInMemoryBufferCache memBufferCache)
+            throws IndexException {
+        return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(memBufferCache, memFreePageManager,
+                invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory);
+    }
+
+}
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
index 41dc7d8..ceae345 100644
--- a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java
@@ -100,10 +100,6 @@
 
     @Override
     public void pinPages() throws HyracksDataException, IndexException {
-        if (cursorNeedsClose) {
-            // Cursor has already been positioned.
-            return;
-        }
         btreePred.setLowKeyComparator(tokenFieldsCmp);
         btreePred.setHighKeyComparator(tokenFieldsCmp);
         btreePred.setLowKey(tokenTuple, true);
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 43bf439..21fc0fb 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
@@ -15,6 +15,7 @@
 package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory;
 
 import java.util.ArrayList;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -40,6 +41,7 @@
 
 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;
 
@@ -67,13 +69,15 @@
         maxPartitionIndex = Integer.MIN_VALUE;
     }
 
-    public synchronized void updatePartitionIndexes(int numTokens) {
+    public void updatePartitionIndexes(int numTokens) {
+        partitionIndexLock.writeLock().lock();
         if (numTokens < minPartitionIndex) {
             minPartitionIndex = numTokens;
         }
         if (numTokens > maxPartitionIndex) {
             maxPartitionIndex = numTokens;
         }
+        partitionIndexLock.writeLock().unlock();
     }
 
     @Override
@@ -87,6 +91,13 @@
     public void openInvertedListPartitionCursors(IInvertedIndexSearcher searcher, IIndexOperationContext ictx,
             int numTokensLowerBound, int numTokensUpperBound, InvertedListPartitions invListPartitions)
             throws HyracksDataException, IndexException {
+        int minPartitionIndex;
+        int maxPartitionIndex;
+        partitionIndexLock.readLock().lock();
+        minPartitionIndex = this.minPartitionIndex;
+        maxPartitionIndex = this.maxPartitionIndex;
+        partitionIndexLock.readLock().unlock();
+
         if (minPartitionIndex == Integer.MAX_VALUE || maxPartitionIndex == Integer.MIN_VALUE) {
             // Index must be empty.
             return;
@@ -116,21 +127,15 @@
                     .getCachedInvertedListCursor();
             inMemListCursor.prepare(ctx.btreeAccessor, ctx.btreePred, ctx.tokenFieldsCmp, ctx.btreeCmp);
             inMemListCursor.reset(searchKey);
-            inMemListCursor.pinPages();
-            if (inMemListCursor.hasNext()) {
-                invListPartitions.addInvertedListCursor(inMemListCursor, i);
-                // Leave the cursor "pinned" to avoid re-traversing the underlying BTree when merging the inverted lists.
-            } else {
-                inMemListCursor.unpinPages();
-            }
+            invListPartitions.addInvertedListCursor(inMemListCursor, i);
         }
     }
 
     @Override
     public void cleanupPartitionState(ArrayList<IInvertedListCursor> invListCursors) throws HyracksDataException {
         // Make sure to unpin/close cursors if the entire partition is pruned.
-        for (IInvertedListCursor cursor : invListCursors) {
-            cursor.unpinPages();
-        }
+        //for (IInvertedListCursor cursor : invListCursors) {
+        //    cursor.unpinPages();
+        //}
     }
 }
diff --git a/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
new file mode 100644
index 0000000..854a30f
--- /dev/null
+++ b/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndexFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.ondisk;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexFileNameMapper;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public class PartitionedOnDiskInvertedIndexFactory extends OnDiskInvertedIndexFactory {
+    
+    public PartitionedOnDiskInvertedIndexFactory(IBufferCache bufferCache, IFileMapProvider fileMapProvider,
+            IInvertedListBuilderFactory invListBuilderFactory, ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IInvertedIndexFileNameMapper fileNameMapper) {
+        super(bufferCache, fileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
+                tokenCmpFactories, fileNameMapper);
+    }
+
+    @Override
+    public IInvertedIndex createIndexInstance(FileReference dictBTreeFile) throws IndexException {
+        String invListsFilePath = fileNameMapper.getInvListsFilePath(dictBTreeFile.getFile().getPath());
+        FileReference invListsFile = new FileReference(new File(invListsFilePath));
+        IInvertedListBuilder invListBuilder = invListBuilderFactory.create();
+        return new PartitionedOnDiskInvertedIndex(bufferCache, fileMapProvider, invListBuilder, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, dictBTreeFile, invListsFile);
+    }
+}
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 1a7bffa..953d72b 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
@@ -43,6 +43,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexFileManager;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.PartitionedLSMInvertedIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.inmemory.PartitionedInMemoryInvertedIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeElementInvertedListBuilder;
@@ -50,6 +51,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndexFactory;
 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;
@@ -98,14 +100,9 @@
         return new FileReference(new File(invListsFile.getFile().getPath() + "_btree"));
     }
 
-    public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
-            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+    public static BTreeFactory createDeletedKeysBTreeFactory(IFileMapProvider diskFileMapProvider,
             ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
-            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
-            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
-            String onDiskDir, ILSMFlushController flushController, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler) throws IndexException {
-
+            IBufferCache diskBufferCache) throws BTreeException {
         TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(invListTypeTraits);
         ITreeIndexFrameFactory leafFrameFactory = BTreeUtils.getLeafFrameFactory(tupleWriterFactory,
                 BTreeLeafFrameType.REGULAR_NSM);
@@ -116,6 +113,19 @@
         BTreeFactory deletedKeysBTreeFactory = new BTreeFactory(diskBufferCache, diskFileMapProvider,
                 freePageManagerFactory, interiorFrameFactory, leafFrameFactory, invListCmpFactories,
                 invListCmpFactories.length);
+        return deletedKeysBTreeFactory;
+    }
+
+    public static LSMInvertedIndex createLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
+            String onDiskDir, ILSMFlushController flushController, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler) throws IndexException {
+
+        BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, diskBufferCache);
 
         FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
         LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
@@ -133,4 +143,32 @@
                 ioScheduler);
         return invIndex;
     }
+
+    public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IInMemoryBufferCache memBufferCache,
+            IInMemoryFreePageManager memFreePageManager, IFileMapProvider diskFileMapProvider,
+            ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
+            ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+            IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, IIOManager ioManager,
+            String onDiskDir, ILSMFlushController flushController, ILSMMergePolicy mergePolicy,
+            ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationScheduler ioScheduler) throws IndexException {
+
+        BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, diskBufferCache);
+
+        FileReference onDiskDirFileRef = new FileReference(new File(onDiskDir));
+        LSMInvertedIndexFileManager fileManager = new LSMInvertedIndexFileManager(ioManager, diskFileMapProvider,
+                onDiskDirFileRef, deletedKeysBTreeFactory);
+
+        IInvertedListBuilderFactory invListBuilderFactory = new FixedSizeElementInvertedListBuilderFactory(
+                invListTypeTraits);
+        PartitionedOnDiskInvertedIndexFactory invIndexFactory = new PartitionedOnDiskInvertedIndexFactory(
+                diskBufferCache, diskFileMapProvider, invListBuilderFactory, invListTypeTraits, invListCmpFactories,
+                tokenTypeTraits, tokenCmpFactories, fileManager);
+
+        PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
+                invIndexFactory, deletedKeysBTreeFactory, fileManager, diskFileMapProvider, invListTypeTraits,
+                invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, flushController,
+                mergePolicy, opTrackerFactory, ioScheduler);
+        return invIndex;
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
new file mode 100644
index 0000000..f7a36f0
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexBulkLoadTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexBulkLoadTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedLSMInvertedIndexBulkLoadTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexDeleteTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexDeleteTest.java
new file mode 100644
index 0000000..4fd529b
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexDeleteTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexDeleteTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexDeleteTest extends AbstractInvertedIndexDeleteTest {
+
+    public PartitionedLSMInvertedIndexDeleteTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, false);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
new file mode 100644
index 0000000..4608f81
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexInsertTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexInsertTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedLSMInvertedIndexInsertTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, false, 1);
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
new file mode 100644
index 0000000..52bb3f8
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleGenerator;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils;
+
+public class PartitionedLSMInvertedIndexMergeTest extends AbstractInvertedIndexLoadTest {
+
+    private final int maxTreesToMerge = AccessMethodTestsConfig.LSM_INVINDEX_MAX_TREES_TO_MERGE;
+    
+    public PartitionedLSMInvertedIndexMergeTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, true, 1);
+    }
+
+    @Override
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen) throws IOException,
+            IndexException {
+        IIndex invIndex = testCtx.getIndex();        
+        invIndex.create();
+        invIndex.activate();
+        ILSMIndexAccessor invIndexAccessor = (ILSMIndexAccessor) invIndex.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+
+        for (int i = 0; i < maxTreesToMerge; i++) {
+            for (int j = 0; j < i; j++) {
+                if (bulkLoad) {
+                    LSMInvertedIndexTestUtils.bulkLoadInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+                } else {
+                    LSMInvertedIndexTestUtils.insertIntoInvIndex(testCtx, tupleGen, NUM_DOCS_TO_INSERT);
+                }
+            }
+            // Perform merge.
+            ILSMIOOperation ioop = invIndexAccessor.createMergeOperation(NoOpIOOperationCallback.INSTANCE);
+            if (ioop != null) {
+                invIndexAccessor.merge(ioop);
+            }
+            validateAndCheckIndex(testCtx);
+            runTinySearchWorkload(testCtx, tupleGen);
+        }
+        
+        invIndex.deactivate();
+        invIndex.destroy();
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
new file mode 100644
index 0000000..80a3c0b
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMultiBulkLoadTest.java
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexLoadTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexMultiBulkLoadTest extends AbstractInvertedIndexLoadTest {
+
+    public PartitionedLSMInvertedIndexMultiBulkLoadTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, true, AccessMethodTestsConfig.LSM_INVINDEX_NUM_BULKLOAD_ROUNDS);
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexSearchTest.java
new file mode 100644
index 0000000..c8a7667
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexSearchTest.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.common.AbstractInvertedIndexSearchTest;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexSearchTest extends AbstractInvertedIndexSearchTest {
+
+    public PartitionedLSMInvertedIndexSearchTest() {
+        super(InvertedIndexType.PARTITIONED_LSM, false);
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTest.java
index 8b01a40..bd48068 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTest.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexMultiThreadTest.java
@@ -39,17 +39,17 @@
 public class LSMInvertedIndexMultiThreadTest {
 
     protected final Logger LOGGER = Logger.getLogger(LSMInvertedIndexMultiThreadTest.class.getName());
-    
+
     // Machine-specific number of threads to use for testing.
     protected final int REGULAR_NUM_THREADS = Runtime.getRuntime().availableProcessors();
     // Excessive number of threads for testing.
     protected final int EXCESSIVE_NUM_THREADS = Runtime.getRuntime().availableProcessors() * 4;
     protected final int NUM_OPERATIONS = AccessMethodTestsConfig.LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS;
-    
-    private final LSMInvertedIndexTestHarness harness = new LSMInvertedIndexTestHarness();
-    private final LSMInvertedIndexWorkerFactory workerFactory = new LSMInvertedIndexWorkerFactory();
-    private final ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
-    
+
+    protected final LSMInvertedIndexTestHarness harness = new LSMInvertedIndexTestHarness();
+    protected final LSMInvertedIndexWorkerFactory workerFactory = new LSMInvertedIndexWorkerFactory();
+    protected final ArrayList<TestWorkloadConf> workloadConfs = getTestWorkloadConf();
+
     protected void setUp() throws HyracksException {
         harness.setUp();
     }
@@ -58,8 +58,8 @@
         harness.tearDown();
     }
 
-    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen, int numThreads, TestWorkloadConf conf,
-            String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
+    protected void runTest(LSMInvertedIndexTestContext testCtx, TupleGenerator tupleGen, int numThreads,
+            TestWorkloadConf conf, String dataMsg) throws InterruptedException, TreeIndexException, HyracksException {
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("LSMInvertedIndex MultiThread Test:\nData: " + dataMsg + "; Threads: " + numThreads
                     + "; Workload: " + conf.toString() + ".");
@@ -123,23 +123,23 @@
 
         return workloadConfs;
     }
-    
+
     @Test
     public void wordTokensInvIndexTest() throws IOException, IndexException, InterruptedException {
         String dataMsg = "Documents";
         int[] numThreads = new int[] { REGULAR_NUM_THREADS, EXCESSIVE_NUM_THREADS };
         for (int i = 0; i < numThreads.length; i++) {
-            for (TestWorkloadConf conf : workloadConfs) {                
+            for (TestWorkloadConf conf : workloadConfs) {
                 setUp();
                 LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createWordInvIndexTestContext(harness,
-                        InvertedIndexType.LSM);
+                        getIndexType());
                 TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createStringDocumentTupleGen(harness.getRandom());
                 runTest(testCtx, tupleGen, numThreads[i], conf, dataMsg);
                 tearDown();
             }
         }
     }
-    
+
     @Test
     public void hashedNGramTokensInvIndexTest() throws IOException, IndexException, InterruptedException {
         String dataMsg = "Person Names";
@@ -148,11 +148,15 @@
             for (TestWorkloadConf conf : workloadConfs) {
                 setUp();
                 LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestUtils.createHashedNGramInvIndexTestContext(
-                        harness, InvertedIndexType.LSM);
+                        harness, getIndexType());
                 TupleGenerator tupleGen = LSMInvertedIndexTestUtils.createPersonNamesTupleGen(harness.getRandom());
                 runTest(testCtx, tupleGen, numThreads[i], conf, dataMsg);
                 tearDown();
             }
         }
     }
+
+    protected InvertedIndexType getIndexType() {
+        return InvertedIndexType.LSM;
+    }
 }
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/PartitionedLSMInvertedIndexMultiThreadTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/PartitionedLSMInvertedIndexMultiThreadTest.java
new file mode 100644
index 0000000..bc6964f
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/PartitionedLSMInvertedIndexMultiThreadTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.multithread;
+
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext.InvertedIndexType;
+
+public class PartitionedLSMInvertedIndexMultiThreadTest extends LSMInvertedIndexMultiThreadTest {
+
+    protected final Logger LOGGER = Logger.getLogger(PartitionedLSMInvertedIndexMultiThreadTest.class.getName());
+    
+    protected InvertedIndexType getIndexType() {
+        return InvertedIndexType.PARTITIONED_LSM;
+    }
+    
+}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 4edb1f6..22c010d 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -141,8 +141,7 @@
                         tokenCmpFactories, harness.getInvListsFileRef());
                 break;
             }
-            case LSM:
-            case PARTITIONED_LSM: {
+            case LSM: {
                 invIndex = InvertedIndexUtils.createLSMInvertedIndex(harness.getMemBufferCache(),
                         harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
                         invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
@@ -151,6 +150,15 @@
                         harness.getIOScheduler());
                 break;
             }
+            case PARTITIONED_LSM: {
+                invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(harness.getMemBufferCache(),
+                        harness.getMemFreePageManager(), harness.getDiskFileMapProvider(), invListTypeTraits,
+                        invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+                        harness.getDiskBufferCache(), harness.getIOManager(), harness.getOnDiskDir(),
+                        harness.getFlushController(), harness.getMergePolicy(), harness.getOperationTrackerFactory(),
+                        harness.getIOScheduler());
+                break;
+            }
             default: {
                 throw new InvertedIndexException("Unknow inverted-index type '" + invIndexType + "'.");
             }