Added multi-threading test framework for tree indexes. Still need to implement concrete tests.

git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_lsm_tree@1113 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 329e380..f531ae6 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -99,7 +99,7 @@
             lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getComparatorFactories(), lowKey);
             highKeySearchCmp = BTreeUtils
                     .getSearchMultiComparator(btree.getComparatorFactories(), highKey);
-            rangePred = new RangePredicate(null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
+            rangePred = new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
                     highKeySearchCmp);
 
             writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
@@ -151,8 +151,6 @@
                 if (highKey != null) {
                     highKey.reset(accessor, i);
                 }
-                rangePred.setLowKey(lowKey, lowKeyInclusive);
-                rangePred.setHighKey(highKey, highKeyInclusive);
                 cursor.reset();
                 indexAccessor.search(cursor, rangePred);
                 writeSearchResults();
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
index 4770307..8c8c12f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java
@@ -91,7 +91,7 @@
     @Override
     public void insert(ITupleReference tuple, int tupleIndex) {
         int freeSpace = buf.getInt(freeSpaceOff);
-        slotManager.insertSlot(tupleIndex, freeSpace);        
+        slotManager.insertSlot(tupleIndex, freeSpace);
         int bytesWritten = tupleWriter.writeTuple(tuple, buf.array(), freeSpace);
         buf.putInt(tupleCountOff, buf.getInt(tupleCountOff) + 1);
         buf.putInt(freeSpaceOff, buf.getInt(freeSpaceOff) + bytesWritten);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
index 50934de..7d5e87f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java
@@ -549,14 +549,14 @@
                             case UPDATE: {
                                 // Is there a propagated split key?
                                 if (ctx.splitKey.getBuffer() != null) {
-                                    node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
-                                    node.acquireWriteLatch();
+                                    ICachedPage interiorNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+                                    interiorNode.acquireWriteLatch();
                                     try {
                                         // Insert or update op. Both can cause split keys to propagate upwards.                                            
-                                        insertInterior(node, pageId, ctx.splitKey.getTuple(), ctx);
+                                        insertInterior(interiorNode, pageId, ctx.splitKey.getTuple(), ctx);
                                     } finally {
-                                        node.releaseWriteLatch();
-                                        bufferCache.unpin(node);
+                                    	interiorNode.releaseWriteLatch();
+                                        bufferCache.unpin(interiorNode);
                                     }
                                 } else {
                                     unsetSmPages(ctx);
@@ -628,15 +628,19 @@
             }
         } catch (TreeIndexException e) {
         	if (!ctx.exceptionHandled) {
-                releaseLatch(node, ctx, unsafeIsLeaf);
-                bufferCache.unpin(node);
-                ctx.exceptionHandled = true;
+        		if (node != null) {
+        			releaseLatch(node, ctx, unsafeIsLeaf);
+        			bufferCache.unpin(node);
+        			ctx.exceptionHandled = true;
+        		}
             }
             throw e;
         } catch (Exception e) {
         	e.printStackTrace();
-            releaseLatch(node, ctx, unsafeIsLeaf);
-            bufferCache.unpin(node);
+        	if (node != null) {
+        		releaseLatch(node, ctx, unsafeIsLeaf);
+        		bufferCache.unpin(node);
+        	}
             BTreeException wrappedException = new BTreeException(e);
             ctx.exceptionHandled = true;
             throw wrappedException;
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
index 150ca2e..db753e3 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/datagen/DataGenThread.java
@@ -22,7 +22,7 @@
     private TupleBatch[] tupleBatches;
     private int ringPos;
     
-    public DataGenThread(int maxNumBatches, int batchSize, int maxOutstandingBatches, int numConsumers, ISerializerDeserializer[] fieldSerdes, int payloadSize, int rndSeed, boolean sorted) {
+    public DataGenThread(int numConsumers, int maxNumBatches, int batchSize, ISerializerDeserializer[] fieldSerdes, int payloadSize, int rndSeed, int maxOutstandingBatches, boolean sorted) {
         this.maxNumBatches = maxNumBatches;
         this.maxOutstandingBatches = maxOutstandingBatches;
         rnd = new Random(rndSeed);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/AbstractTreeIndexTestWorker.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/AbstractTreeIndexTestWorker.java
new file mode 100644
index 0000000..4e853f3
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/AbstractTreeIndexTestWorker.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.test;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.datagen.TupleBatch;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector.TestOperation;
+
+public abstract class AbstractTreeIndexTestWorker extends Thread implements ITreeIndexTestWorker {
+    private Random rnd = new Random();
+    private final DataGenThread dataGen;
+    private final TestOperationSelector opSelector;
+    private final int numBatches;
+    
+    protected final ITreeIndexAccessor indexAccessor;
+    
+    public AbstractTreeIndexTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches) {
+        this.dataGen = dataGen;
+        this.opSelector = opSelector;
+        this.numBatches = numBatches;
+        indexAccessor = index.createAccessor();
+    }
+    
+    @Override
+    public void run() {
+        try {
+            for (int i = 0; i < numBatches; i++) {
+                TupleBatch batch = dataGen.tupleBatchQueue.take();
+                for (int j = 0; j < batch.size(); j++) {
+                    TestOperation op = opSelector.getOp(rnd.nextInt());
+                    ITupleReference tuple = batch.get(j);
+                    performOp(tuple, op);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/ITreeIndexTestWorker.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/ITreeIndexTestWorker.java
new file mode 100644
index 0000000..a12bf73
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/ITreeIndexTestWorker.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector.TestOperation;
+
+public interface ITreeIndexTestWorker {
+    void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, TreeIndexException;
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/ITreeIndexTestWorkerFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/ITreeIndexTestWorkerFactory.java
new file mode 100644
index 0000000..d0ee2b4
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/ITreeIndexTestWorkerFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.test;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+
+public interface ITreeIndexTestWorkerFactory {
+    public AbstractTreeIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches);
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/TestOperationSelector.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/TestOperationSelector.java
new file mode 100644
index 0000000..25058a9
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/TestOperationSelector.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.test;
+
+import java.util.Arrays;
+
+public class TestOperationSelector {
+
+    public static enum TestOperation {
+        INSERT,
+        DELETE,
+        UPDATE,
+        POINT_SEARCH,
+        RANGE_SEARCH,
+        ORDERED_SCAN,
+        DISKORDER_SCAN,
+        FLUSH,
+        MERGE        
+    }
+    
+    private final TestOperation[] ops;
+    private final int[] opRanges;    
+    
+    public TestOperationSelector(TestOperation[] ops, float[] opProbs) {
+        sanityCheck(ops, opProbs);
+        this.ops = ops;
+        this.opRanges = getOpRanges(opProbs);
+    }
+    
+    private void sanityCheck(TestOperation[] ops, float[] opProbs) {
+        if (ops.length == 0) {
+            throw new RuntimeException("Empty op array.");
+        }
+        if (opProbs.length == 0) {
+            throw new RuntimeException("Empty op probabilities.");
+        }
+        if (ops.length != opProbs.length) {
+            throw new RuntimeException("Ops and op probabilities have unequal length.");
+        }
+        float sum = 0.0f;
+        for (int i = 0; i < opProbs.length; i++) {
+            sum += opProbs[i];
+        }
+        if (sum != 1.0f) {
+            throw new RuntimeException("Op probabilities don't add up to 1.");
+        }
+    }
+    
+    private int[] getOpRanges(float[] opProbabilities) {
+        int[] opRanges = new int[opProbabilities.length];
+        if (opRanges.length > 1) {
+            opRanges[0] = (int) Math.floor(Integer.MAX_VALUE * opProbabilities[0]);
+            for (int i = 1; i < opRanges.length - 1; i++) {
+                opRanges[i] = opRanges[i - 1] + (int) Math.floor(Integer.MAX_VALUE * opProbabilities[i]);
+            }
+            opRanges[opRanges.length - 1] = Integer.MAX_VALUE;
+        } else {
+            opRanges[0] = Integer.MAX_VALUE;
+        }
+        return opRanges;
+    }
+    
+    public TestOperation getOp(int randomInt) {
+        int ix = Arrays.binarySearch(opRanges, randomInt);
+        if (ix < 0) {
+            ix = -ix - 1;
+        }
+        return ops[ix];
+    }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/TreeIndexMultiThreadTestDriver.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/TreeIndexMultiThreadTestDriver.java
new file mode 100644
index 0000000..e0b6b47
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/test/TreeIndexMultiThreadTestDriver.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.common.test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector.TestOperation;
+
+@SuppressWarnings("rawtypes")
+public class TreeIndexMultiThreadTestDriver {
+    private static final int RANDOM_SEED = 50;
+    private static final int MAX_OUTSTANDING_BATCHES = 10;
+    // Means no additional payload. Only the specified fields.
+    private static final int PAYLOAD_SIZE = 0;
+    private final TestOperationSelector opSelector;    
+    private final ISerializerDeserializer[] fieldSerdes;
+    private final ITreeIndex index;
+    private final ITreeIndexTestWorkerFactory workerFactory;
+    
+    public TreeIndexMultiThreadTestDriver(ITreeIndex index, ITreeIndexTestWorkerFactory workerFactory,
+            ISerializerDeserializer[] fieldSerdes, TestOperation[] ops, float[] opProbs) {
+        this.index = index;
+        this.workerFactory = workerFactory;
+        this.fieldSerdes = fieldSerdes;
+        this.opSelector = new TestOperationSelector(ops, opProbs);
+    }
+    
+    public void run(int numThreads, int numRepeats, int numTuples, int batchSize) throws InterruptedException, TreeIndexException {
+        int numBatches = numTuples / batchSize;
+        int threadNumBatches = numBatches / numThreads;
+        if (threadNumBatches <= 0) {
+            throw new TreeIndexException("Inconsistent parameters given. Need at least one batch per thread.");
+        }
+        long[] times = new long[numRepeats];
+        for (int i = 0; i < numRepeats; i++) {
+            DataGenThread dataGen = createDatagenThread(numThreads, numBatches, batchSize);
+            dataGen.start();
+            // Wait until the tupleBatchQueue is filled to capacity.
+            while (dataGen.tupleBatchQueue.remainingCapacity() != 0 && dataGen.tupleBatchQueue.size() != numBatches) {
+                Thread.sleep(10);
+            }
+                        
+            // Start worker threads.
+            AbstractTreeIndexTestWorker[] workers = new AbstractTreeIndexTestWorker[numThreads];
+            long start = System.currentTimeMillis();
+            for (int j = 0; j < numThreads; j++) {
+                workers[j] = workerFactory.create(dataGen, opSelector, index, threadNumBatches);
+                workers[j].start();
+            }
+            // Join worker threads.
+            for (int j = 0; j < numThreads; j++) {
+                workers[j].join();
+            }
+            long end = System.currentTimeMillis();
+            times[i] = end - start;
+        }
+    }
+    
+    // To allow subclasses to override the data gen params.
+    public DataGenThread createDatagenThread(int numThreads, int numBatches, int batchSize) {
+        return new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, PAYLOAD_SIZE, RANDOM_SEED, MAX_OUTSTANDING_BATCHES, false);
+    }
+}
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index a5584e0..1d2b091 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -37,7 +37,8 @@
     private static final Logger LOGGER = Logger.getLogger(BufferCache.class.getName());
     private static final int MAP_FACTOR = 2;
 
-    private static final int MAX_VICTIMIZATION_TRY_COUNT = 3;
+    //private static final int MAX_VICTIMIZATION_TRY_COUNT = 3;
+    private static final int MAX_VICTIMIZATION_TRY_COUNT = 1000000;
 
     private final int maxOpenFiles;
 
@@ -137,7 +138,8 @@
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info(dumpState());
             }
-            throw new HyracksDataException("Failed to pin page because all pages are pinned.");
+            throw new HyracksDataException("Failed to pin page " + BufferedFileHandle.getFileId(dpid) + ":"
+                    + BufferedFileHandle.getPageId(dpid) + " because all pages are pinned.");
         }
         if (!newPage) {
             if (!cPage.valid) {
@@ -380,6 +382,9 @@
 
     private void write(CachedPage cPage) throws HyracksDataException {
         BufferedFileHandle fInfo = getFileInfo(cPage);
+        if(fInfo.fileHasBeenDeleted()){
+            return;
+        }
         cPage.buffer.position(0);
         cPage.buffer.limit(pageSize);
         ioManager.syncWrite(fInfo.getFileHandle(), (long) BufferedFileHandle.getPageId(cPage.dpid) * pageSize,
@@ -525,7 +530,7 @@
                                         cPage.pinCount.decrementAndGet();
                                         synchronized (cleanNotification) {
                                             ++cleanCount;
-                                            cleanNotification.notifyAll();
+                                            cleanNotification.notify();
                                         }
                                     }
                                 } finally {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
new file mode 100644
index 0000000..15d4d73
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeMultiThreadTest.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.btree.multithread;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector.TestOperation;
+import edu.uci.ics.hyracks.storage.am.common.test.TreeIndexMultiThreadTestDriver;
+
+@SuppressWarnings("rawtypes")
+public class BTreeMultiThreadTest {    
+    
+    private final int PAGE_SIZE = 8192;
+    private final int NUM_PAGES = 1000;
+    private final int MAX_OPEN_FILES = 10;
+    private final int HYRACKS_FRAME_SIZE = 32768;
+    
+    private final BTreeTestWorkerFactory workerFactory = new BTreeTestWorkerFactory();
+    private final BTreeTestHarness harness = new BTreeTestHarness(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES, HYRACKS_FRAME_SIZE);
+
+    @Before
+    public void setUp() throws HyracksDataException {
+        harness.setUp();
+    }
+
+    @After
+    public void tearDown() throws HyracksDataException {
+        harness.tearDown();
+    }
+    
+    @Test
+    public void firstTest() throws InterruptedException, HyracksDataException, TreeIndexException {        
+        ISerializerDeserializer[] fieldSerdes = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+        ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
+        int numKeyFields = 1;
+        IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);     
+        
+        int btreeFileId = harness.getBTreeFileId();
+        BTree btree = BTreeUtils.createBTree(harness.getBufferCache(), harness.getBTreeFileId(), typeTraits, cmpFactories, BTreeLeafFrameType.REGULAR_NSM);
+        btree.create(btreeFileId);
+        btree.open(btreeFileId);
+        
+        TestOperation[] ops = new TestOperation[] { TestOperation.INSERT };
+        float[] opProbs = new float[] { 1.0f };
+        
+        TreeIndexMultiThreadTestDriver driver = new TreeIndexMultiThreadTestDriver(btree, workerFactory, fieldSerdes, ops, opProbs);
+        
+        //driver.run(10, 1, 10000000, 1000);
+        driver.run(6, 1, 10000000, 1000);
+        
+        btree.close();
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
new file mode 100644
index 0000000..7896394
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.btree.multithread;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.test.AbstractTreeIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector.TestOperation;
+
+public class BTreeTestWorker extends AbstractTreeIndexTestWorker {
+    
+    public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, ITreeIndex index, int numBatches) {
+        super(dataGen, opSelector, index, numBatches);
+    }
+    
+    @Override
+    public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, TreeIndexException {
+        switch (op) {
+            case INSERT:
+                try {
+                    indexAccessor.insert(tuple);
+                } catch (BTreeDuplicateKeyException e) {
+                    // Ignore duplicate keys.
+                }
+                break;
+                
+            
+            default:
+                throw new HyracksDataException("Op " + op.toString() + " not supported.");
+        }
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
new file mode 100644
index 0000000..e954aba
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorkerFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.storage.am.btree.multithread;
+
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.datagen.DataGenThread;
+import edu.uci.ics.hyracks.storage.am.common.test.AbstractTreeIndexTestWorker;
+import edu.uci.ics.hyracks.storage.am.common.test.ITreeIndexTestWorkerFactory;
+import edu.uci.ics.hyracks.storage.am.common.test.TestOperationSelector;
+
+public class BTreeTestWorkerFactory implements ITreeIndexTestWorkerFactory {
+    @Override
+    public AbstractTreeIndexTestWorker create(DataGenThread dataGen, TestOperationSelector opSelector,
+            ITreeIndex index, int numBatches) {
+        return new BTreeTestWorker(dataGen, opSelector, index, numBatches);
+    }
+}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java
index 3bf128e..59bd31d 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestHarness.java
@@ -35,7 +35,7 @@
     
     private static final long RANDOM_SEED = 50;
     private static final int DEFAULT_PAGE_SIZE = 256;
-    private static final int DEFAULT_NUM_PAGES = 10;
+    private static final int DEFAULT_NUM_PAGES = 100;
     private static final int DEFAULT_MAX_OPEN_FILES = 10;
     private static final int DEFAULT_HYRACKS_FRAME_SIZE = 128;
     
diff --git a/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java b/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
index 079a4d5..7e0514b 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/BTreePageSizePerf.java
@@ -68,7 +68,7 @@
         runner.init();
         int numThreads = 1;
         for (int i = 0; i < repeats; i++) {
-            DataGenThread dataGen = new DataGenThread(numBatches, batchSize, 10, numThreads, fieldSerdes, 30, 50, false);
+            DataGenThread dataGen = new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, 30, 50, 10, false);
             dataGen.start();            
             times[i] = runner.runExperiment(dataGen, numThreads);
             System.out.println("TIME " + i + ": " + times[i] + "ms");
diff --git a/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java b/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
index a9548e4..c842191 100644
--- a/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
+++ b/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/perf/PerfExperiment.java
@@ -70,7 +70,7 @@
         	//BTreeSearchRunner runner = new BTreeSearchRunner(btreeName, 10, numBatches, 8192, 25000, typeTraits, cmp);
         	//LSMTreeRunner runner = new LSMTreeRunner(numBatches, 8192, 100, 8192, 250, typeTraits, cmp);
         	//LSMTreeSearchRunner runner = new LSMTreeSearchRunner(100000, numBatches, 8192, 24750, 8192, 250, typeTraits, cmp); 
-            DataGenThread dataGen = new DataGenThread(numBatches, batchSize, 10, numThreads, fieldSerdes, 30, 50, false);
+            DataGenThread dataGen = new DataGenThread(numThreads, numBatches, batchSize, fieldSerdes, 30, 50, 10, false);
             dataGen.start();
             runner.reset();
             times[i] = runner.runExperiment(dataGen, numThreads);