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);