Reintegrated hyracks_btree_updates_next.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_dev_next@831 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java
index 032fddf..171dbb7 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/marshalling/IntegerSerializerDeserializer.java
@@ -51,4 +51,11 @@
return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+ ((bytes[offset + 3] & 0xff) << 0);
}
+
+ public static void putInt(int val, byte[] bytes, int offset) {
+ bytes[offset] = (byte)((val >>> 24) & 0xFF);
+ bytes[offset + 1] = (byte)((val >>> 16) & 0xFF);
+ bytes[offset + 2] = (byte)((val >>> 8) & 0xFF);
+ bytes[offset + 3] = (byte)((val >>> 0) & 0xFF);
+ }
}
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
index 1e9c47f..14c4d66 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/TupleUtils.java
@@ -33,7 +33,8 @@
public static void createTuple(ArrayTupleBuilder tupleBuilder, ArrayTupleReference tuple, ISerializerDeserializer[] fieldSerdes, final Object... fields) throws HyracksDataException {
DataOutput dos = tupleBuilder.getDataOutput();
tupleBuilder.reset();
- for (int i = 0; i < fields.length; i++) {
+ int numFields = Math.min(tupleBuilder.getFieldEndOffsets().length, fields.length);
+ for (int i = 0; i < numFields; i++) {
fieldSerdes[i].serialize(fields[i], dos);
tupleBuilder.addFieldEndOffset();
}
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 1b90300..4711e9e 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -41,15 +41,15 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
@@ -128,7 +128,7 @@
// run data generator on first nodecontroller given
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dataGen, splitNCs[0]);
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
+ IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
// prepare insertion into primary index
@@ -156,13 +156,13 @@
// B-Tree tuple, etc.
IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
- ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
// create operator descriptor
TreeIndexInsertUpdateDeleteOperatorDescriptor primaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recDesc, storageManager, treeIndexRegistryProvider, primarySplitProvider,
+ spec, recDesc, storageManager, indexRegistryProvider, primarySplitProvider,
primaryInteriorFrameFactory, primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories,
- primaryFieldPermutation, IndexOp.INSERT, opHelperFactory);
+ primaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, primaryInsert, splitNCs);
// prepare insertion into secondary index
@@ -190,9 +190,9 @@
options.secondaryBTreeName);
// create operator descriptor
TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, recDesc, storageManager, treeIndexRegistryProvider, secondarySplitProvider,
+ spec, recDesc, storageManager, indexRegistryProvider, secondarySplitProvider,
secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryTypeTraits,
- secondaryComparatorFactories, secondaryFieldPermutation, IndexOp.INSERT, opHelperFactory);
+ secondaryComparatorFactories, secondaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, secondaryInsert, splitNCs);
// end the insert pipeline at this sink operator
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 4656a88..9790496 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -39,15 +39,15 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.examples.btree.helper.DataGenOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -147,7 +147,7 @@
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
+ IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
// the B-Tree expects its keyfields to be at the front of its input
@@ -156,10 +156,10 @@
// to field 0 of B-Tree tuple,
// etc.
IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
- ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, treeIndexRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory,
- typeTraits, comparatorFactories, fieldPermutation, 0.7f, opHelperFactory);
+ storageManager, indexRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory,
+ typeTraits, comparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
// distribute the records from the datagen via hashing to the bulk load
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
deleted file mode 100644
index ae55700..0000000
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * 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.examples.btree.client;
-
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-
-import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-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.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexFileEnlistmentOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-// This example will enlist existing files as primary index
-
-public class PrimaryIndexEnlistFilesExample {
- private static class Options {
- @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
- public String host;
-
- @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1099)")
- public int port = 1099;
-
- @Option(name = "-app", usage = "Hyracks Application name", required = true)
- public String app;
-
- @Option(name = "-target-ncs", usage = "Comma separated list of node-controller names to use", required = true)
- public String ncs;
-
- @Option(name = "-btreename", usage = "B-Tree file name", required = true)
- public String btreeName;
- }
-
- public static void main(String[] args) throws Exception {
- Options options = new Options();
- CmdLineParser parser = new CmdLineParser(options);
- parser.parseArgument(args);
-
- IHyracksClientConnection hcc = new HyracksRMIConnection(options.host, options.port);
-
- JobSpecification job = createJob(options);
-
- long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
- hcc.waitForCompletion(jobId);
- long end = System.currentTimeMillis();
- System.err.println(start + " " + end + " " + (end - start));
- }
-
- private static JobSpecification createJob(Options options) {
-
- JobSpecification spec = new JobSpecification();
-
- String[] splitNCs = options.ncs.split(",");
-
- // schema of tuples in existing files (see PrimaryIndexBulkLoadExample)
- RecordDescriptor recDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
- int fieldCount = 4;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- typeTraits[2] = new TypeTrait(4);
- typeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-
- IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
- comparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-
- // create factories and providers for B-Tree
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
- IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
-
- IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
- ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
- TreeIndexFileEnlistmentOperatorDescriptor fileEnlistmentOp = new TreeIndexFileEnlistmentOperatorDescriptor(
- spec, recDesc, storageManager, treeIndexRegistryProvider, btreeSplitProvider, interiorFrameFactory,
- leafFrameFactory, typeTraits, comparatorFactories, opHelperFactory);
- JobHelper.createPartitionConstraint(spec, fileEnlistmentOp, splitNCs);
-
- spec.addRoot(fileEnlistmentOp);
-
- return spec;
- }
-}
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 1d61a0f..4929243 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -37,16 +37,16 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -109,7 +109,7 @@
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
+ IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
// schema of tuples coming out of primary index
@@ -146,10 +146,10 @@
// into search op
IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.btreeName);
- ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
- treeIndexRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
- comparatorFactories, true, lowKeyFields, highKeyFields, true, true, opHelperFactory);
+ indexRegistryProvider, btreeSplitProvider, interiorFrameFactory, leafFrameFactory, typeTraits,
+ comparatorFactories, true, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
// have each node print the results of its respective B-Tree
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index d38568f..d77eb01 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -33,15 +33,15 @@
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDiskOrderScanOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
@@ -97,7 +97,7 @@
String[] splitNCs = options.ncs.split(",");
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
+ IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
// schema of tuples that we are retrieving from the primary index
@@ -129,10 +129,10 @@
// use a disk-order scan to read primary index
IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
- ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
TreeIndexDiskOrderScanOperatorDescriptor btreeScanOp = new TreeIndexDiskOrderScanOperatorDescriptor(spec,
- recDesc, storageManager, treeIndexRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits, opHelperFactory);
+ recDesc, storageManager, indexRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, btreeScanOp, splitNCs);
// sort the tuples as preparation for bulk load into secondary index
@@ -159,9 +159,9 @@
int[] fieldPermutation = { 1, 0 };
IFileSplitProvider btreeSplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.secondaryBTreeName);
TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, treeIndexRegistryProvider, btreeSplitProvider, secondaryInteriorFrameFactory,
+ storageManager, indexRegistryProvider, btreeSplitProvider, secondaryInteriorFrameFactory,
secondaryLeafFrameFactory, secondaryTypeTraits, comparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
+ dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, btreeBulkLoad, splitNCs);
// connect the ops
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 9b5d6fa..3a79df6 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -38,16 +38,16 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.examples.btree.helper.IndexRegistryProvider;
import edu.uci.ics.hyracks.examples.btree.helper.StorageManagerInterface;
-import edu.uci.ics.hyracks.examples.btree.helper.TreeIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -98,7 +98,7 @@
String[] splitNCs = options.ncs.split(",");
- IIndexRegistryProvider<ITreeIndex> btreeRegistryProvider = TreeIndexRegistryProvider.INSTANCE;
+ IIndexRegistryProvider<IIndex> indexRegistryProvider = IndexRegistryProvider.INSTANCE;
IStorageManagerInterface storageManager = StorageManagerInterface.INSTANCE;
// schema of tuples coming out of secondary index
@@ -181,11 +181,11 @@
IFileSplitProvider secondarySplitProvider = JobHelper.createFileSplitProvider(splitNCs,
options.secondaryBTreeName);
- ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
- storageManager, btreeRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
+ storageManager, indexRegistryProvider, secondarySplitProvider, secondaryInteriorFrameFactory,
secondaryLeafFrameFactory, secondaryTypeTraits, searchComparatorFactories, true, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, opHelperFactory);
+ secondaryHighKeyFields, true, true, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
// secondary index will output tuples with [UTF8String, Integer]
@@ -199,9 +199,9 @@
IFileSplitProvider primarySplitProvider = JobHelper.createFileSplitProvider(splitNCs, options.primaryBTreeName);
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, btreeRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
+ storageManager, indexRegistryProvider, primarySplitProvider, primaryInteriorFrameFactory,
primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, primaryLowKeyFields,
- primaryHighKeyFields, true, true, opHelperFactory);
+ primaryHighKeyFields, true, true, dataflowHelperFactory);
JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
// have each node print the results of its respective B-Tree
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexRegistryProvider.java
similarity index 70%
rename from hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
rename to hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexRegistryProvider.java
index f8a7bf1..d1f40d8 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/IndexRegistryProvider.java
@@ -16,20 +16,20 @@
package edu.uci.ics.hyracks.examples.btree.helper;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-public class TreeIndexRegistryProvider implements IIndexRegistryProvider<ITreeIndex> {
+public class IndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
private static final long serialVersionUID = 1L;
- public static final TreeIndexRegistryProvider INSTANCE = new TreeIndexRegistryProvider();
+ public static final IndexRegistryProvider INSTANCE = new IndexRegistryProvider();
- private TreeIndexRegistryProvider() {
+ private IndexRegistryProvider() {
}
@Override
- public IndexRegistry<ITreeIndex> getRegistry(IHyracksTaskContext ctx) {
- return RuntimeContext.get(ctx).getTreeIndexRegistry();
+ public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
+ return RuntimeContext.get(ctx).getIndexRegistry();
}
}
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 4a452a8..3336d6d 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -17,7 +17,7 @@
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -30,7 +30,7 @@
import edu.uci.ics.hyracks.storage.common.smi.TransientFileMapManager;
public class RuntimeContext {
- private IndexRegistry<ITreeIndex> treeIndexRegistry;
+ private IndexRegistry<IIndex> indexRegistry;
private IBufferCache bufferCache;
private IFileMapManager fileMapManager;
@@ -39,7 +39,7 @@
ICacheMemoryAllocator allocator = new HeapBufferAllocator();
IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
bufferCache = new BufferCache(appCtx.getRootContext().getIOManager(), allocator, prs, fileMapManager, 32768, 50, 100);
- treeIndexRegistry = new IndexRegistry<ITreeIndex>();
+ indexRegistry = new IndexRegistry<IIndex>();
}
public void close() {
@@ -54,8 +54,8 @@
return fileMapManager;
}
- public IndexRegistry<ITreeIndex> getTreeIndexRegistry() {
- return treeIndexRegistry;
+ public IndexRegistry<IIndex> getIndexRegistry() {
+ return indexRegistry;
}
public static RuntimeContext get(IHyracksTaskContext ctx) {
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 5ac4af2..38dd051 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -47,196 +46,153 @@
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
public class BTreePrimaryIndexScanOperatorTest extends AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary index
- private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private int primaryKeyFieldCount = 1;
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- primaryTypeTraits);
- private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory);
- private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory);
+ // field, type and key declarations for primary index
+ private int primaryFieldCount = 6;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private int primaryKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
- private static String primaryBtreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryBtreeName;
+ private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
- private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary index
- primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary index
+ primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- loadPrimaryIndexTest();
- }
+ loadPrimaryIndexTest();
+ }
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/tpch0.001/orders-part1.tbl"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/tpch0.001/orders-part1.tbl"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeBulkLoad, NC1_ID);
+ int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- primaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
- spec.addRoot(primaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(primaryBtreeBulkLoad);
+ runTest(spec);
+ }
- @Test
- public void scanPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ @Test
+ public void scanPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- // build dummy tuple containing nothing
- ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
- DataOutput dos = tb.getDataOutput();
+ // build dummy tuple containing nothing
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+ DataOutput dos = tb.getDataOutput();
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+ tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- int[] lowKeyFields = null; // - infinity
- int[] highKeyFields = null; // + infinity
+ int[] lowKeyFields = null; // - infinity
+ int[] highKeyFields = null; // + infinity
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBtreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories, true,
- lowKeyFields, highKeyFields, true, true, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeSearchOp, NC1_ID);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+ highKeyFields, true, true, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
createTempFile().getAbsolutePath()) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- primaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeSearchOp, 0, printer, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
- spec.addRoot(printer);
- runTest(spec);
- }
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
- }
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index d2ee5e4..a043675 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -47,202 +46,158 @@
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-public class BTreePrimaryIndexSearchOperatorTest extends
- AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+public class BTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary index
- private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private int primaryKeyFieldCount = 1;
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- primaryTypeTraits);
- private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory);
- private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory);
+ // field, type and key declarations for primary index
+ private int primaryFieldCount = 6;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private int primaryKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
- private static String primaryBtreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryBtreeName;
+ private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
- private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary index
- primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary index
+ primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- loadPrimaryIndexTest();
- }
+ loadPrimaryIndexTest();
+ }
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/tpch0.001/orders-part1.tbl"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/tpch0.001/orders-part1.tbl"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeBulkLoad, NC1_ID);
+ int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- primaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
- spec.addRoot(primaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(primaryBtreeBulkLoad);
+ runTest(spec);
+ }
- @Test
- public void searchPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ @Test
+ public void searchPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- // build tuple containing low and high search key
- // high key and low key
- ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
- DataOutput dos = tb.getDataOutput();
+ // build tuple containing low and high search key
+ // high key and low key
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+ DataOutput dos = tb.getDataOutput();
- tb.reset();
- // low key
- UTF8StringSerializerDeserializer.INSTANCE.serialize("100", dos);
- tb.addFieldEndOffset();
- // high key
- UTF8StringSerializerDeserializer.INSTANCE.serialize("200", dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ // low key
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("100", dos);
+ tb.addFieldEndOffset();
+ // high key
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("200", dos);
+ tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- int[] lowKeyFields = { 0 };
- int[] highKeyFields = { 1 };
+ int[] lowKeyFields = { 0 };
+ int[] highKeyFields = { 1 };
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBtreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories, true,
- lowKeyFields, highKeyFields, true, true, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeSearchOp, NC1_ID);
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+ highKeyFields, true, true, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
createTempFile().getAbsolutePath()) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- primaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeSearchOp, 0, printer, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
- spec.addRoot(printer);
- runTest(spec);
- }
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
- }
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index e685894..51d3db1 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -24,11 +24,11 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -41,163 +41,132 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexStatsOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
public class BTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary index
- private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private int primaryKeyFieldCount = 1;
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- primaryTypeTraits);
- private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory);
- private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory);
+ // field, type and key declarations for primary index
+ private int primaryFieldCount = 6;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private int primaryKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
- private static String primaryBtreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryBtreeName;
+ private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
- private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary index
- primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary index
+ primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- loadPrimaryIndexTest();
- }
+ loadPrimaryIndexTest();
+ }
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/tpch0.001/orders-part1.tbl"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/tpch0.001/orders-part1.tbl"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeBulkLoad, NC1_ID);
+ int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- primaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
- spec.addRoot(primaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(primaryBtreeBulkLoad);
+ runTest(spec);
+ }
- @Test
- public void showPrimaryIndexStats() throws Exception {
- JobSpecification spec = new JobSpecification();
+ @Test
+ public void showPrimaryIndexStats() throws Exception {
+ JobSpecification spec = new JobSpecification();
- TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryStatsOp, NC1_ID);
+ TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
+ indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory, primaryLeafFrameFactory,
+ primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+ createTempFile().getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.addRoot(primaryStatsOp);
- runTest(spec);
- }
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
- }
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index c2a7201..3b6b202 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -48,404 +47,307 @@
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-public class BTreeSecondaryIndexInsertOperatorTest extends
- AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+public class BTreeSecondaryIndexInsertOperatorTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary index
- private int primaryFieldCount = 6;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private int primaryKeyFieldCount = 1;
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- primaryTypeTraits);
- private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory);
- private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory);
+ // field, type and key declarations for primary index
+ private int primaryFieldCount = 6;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private int primaryKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
- private static String primaryBtreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryBtreeName;
+ private static String primaryBtreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryBtreeName;
- private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- // field, type and key declarations for secondary indexes
- private int secondaryFieldCount = 2;
- private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
- private int secondaryKeyFieldCount = 2;
- private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
- private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
- secondaryTypeTraits);
- private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- secondaryTupleWriterFactory);
- private ITreeIndexFrameFactory secondaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- secondaryTupleWriterFactory);
+ // field, type and key declarations for secondary indexes
+ private int secondaryFieldCount = 2;
+ private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
+ private int secondaryKeyFieldCount = 2;
+ private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory secondaryTupleWriterFactory = new TypeAwareTupleWriterFactory(
+ secondaryTypeTraits);
+ private ITreeIndexFrameFactory secondaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ secondaryTupleWriterFactory);
+ private ITreeIndexFrameFactory secondaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(secondaryTupleWriterFactory);
- private static String secondaryBtreeName = "secondary"
- + simpleDateFormat.format(new Date());
- private static String secondaryFileName = System
- .getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
+ private static String secondaryBtreeName = "secondary" + simpleDateFormat.format(new Date());
+ private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryBtreeName;
- private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- secondaryFileName))) });
+ private IFileSplitProvider secondaryBtreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
- private RecordDescriptor secondaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary index
- primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary index
+ primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- // field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ // field, type and key declarations for secondary indexes
+ secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
- loadPrimaryIndexTest();
- loadSecondaryIndexTest();
- insertPipelineTest();
- }
+ loadPrimaryIndexTest();
+ loadSecondaryIndexTest();
+ insertPipelineTest();
+ }
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/tpch0.001/orders-part1.tbl"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/tpch0.001/orders-part1.tbl"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeBulkLoad, NC1_ID);
+ int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- primaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBtreeBulkLoad, 0);
- spec.addRoot(primaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(primaryBtreeBulkLoad);
+ runTest(spec);
+ }
- public void loadSecondaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ public void loadSecondaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- // build dummy tuple containing nothing
- ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
- DataOutput dos = tb.getDataOutput();
+ // build dummy tuple containing nothing
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+ DataOutput dos = tb.getDataOutput();
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+ tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- int[] lowKeyFields = null; // - infinity
- int[] highKeyFields = null; // + infinity
+ int[] lowKeyFields = null; // - infinity
+ int[] highKeyFields = null; // + infinity
- // scan primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBtreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories, true,
- lowKeyFields, highKeyFields, true, true, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeSearchOp, NC1_ID);
+ // scan primary index
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+ highKeyFields, true, true, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- // sort based on secondary keys
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 3, 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- primaryRecDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ // sort based on secondary keys
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, primaryRecDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- // load secondary index
- int[] fieldPermutation = { 3, 0 };
- TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
- secondaryLeafFrameFactory, secondaryTypeTraits,
- secondaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryBtreeBulkLoad, NC1_ID);
+ // load secondary index
+ int[] fieldPermutation = { 3, 0 };
+ TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
+ secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- primaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeSearchOp, 0, sorter, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- secondaryBtreeBulkLoad, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, sorter, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, secondaryBtreeBulkLoad, 0);
- spec.addRoot(secondaryBtreeBulkLoad);
- runTest(spec);
- }
+ spec.addRoot(secondaryBtreeBulkLoad);
+ runTest(spec);
+ }
- public void insertPipelineTest() throws Exception {
+ public void insertPipelineTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ JobSpecification spec = new JobSpecification();
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/tpch0.001/orders-part2.tbl"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- // insert into primary index
- int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, ordersDesc, storageManager, treeIndexRegistryProvider,
- primaryBtreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, primaryFieldPermutation,
- IndexOp.INSERT, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeInsertOp, NC1_ID);
+ // insert into primary index
+ int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, ordersDesc, storageManager, indexRegistryProvider, primaryBtreeSplitProvider,
+ primaryInteriorFrameFactory, primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories,
+ primaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeInsertOp, NC1_ID);
- // first secondary index
- int[] fieldPermutationB = { 4, 0 };
- TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, ordersDesc, storageManager, treeIndexRegistryProvider,
- secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
- secondaryLeafFrameFactory, secondaryTypeTraits,
- secondaryComparatorFactories, fieldPermutationB,
- IndexOp.INSERT, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryInsertOp, NC1_ID);
+ // first secondary index
+ int[] fieldPermutationB = { 4, 0 };
+ TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
+ spec, ordersDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
+ secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryTypeTraits,
+ secondaryComparatorFactories, fieldPermutationB, IndexOp.INSERT, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryInsertOp, NC1_ID);
- NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(
- spec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, nullSink,
- NC1_ID);
+ NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, nullSink, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- primaryBtreeInsertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, primaryBtreeInsertOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeInsertOp, 0, secondaryInsertOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeInsertOp, 0, secondaryInsertOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), secondaryInsertOp,
- 0, nullSink, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryInsertOp, 0, nullSink, 0);
- spec.addRoot(nullSink);
- runTest(spec);
- }
+ spec.addRoot(nullSink);
+ runTest(spec);
+ }
- @Test
- public void searchUpdatedSecondaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ @Test
+ public void searchUpdatedSecondaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- // build tuple containing search keys (only use the first key as search
- // key)
- ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
- DataOutput dos = tb.getDataOutput();
+ // build tuple containing search keys (only use the first key as search
+ // key)
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+ DataOutput dos = tb.getDataOutput();
- tb.reset();
- // low key
- UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
- tb.addFieldEndOffset();
- // high key
- UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ // low key
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("1998-07-21", dos);
+ tb.addFieldEndOffset();
+ // high key
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("2000-10-18", dos);
+ tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- int[] secondaryLowKeyFields = { 0 };
- int[] secondaryHighKeyFields = { 1 };
+ int[] secondaryLowKeyFields = { 0 };
+ int[] secondaryHighKeyFields = { 1 };
- // search secondary index
- BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, secondaryRecDesc, storageManager,
- treeIndexRegistryProvider, secondaryBtreeSplitProvider,
- secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
- secondaryTypeTraits, secondaryComparatorFactories, true,
- secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryBtreeSearchOp, NC1_ID);
+ // search secondary index
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
+ secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryTypeTraits,
+ secondaryComparatorFactories, true, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
- // second field from the tuples coming from secondary index
- int[] primaryLowKeyFields = { 1 };
- // second field from the tuples coming from secondary index
- int[] primaryHighKeyFields = { 1 };
+ // second field from the tuples coming from secondary index
+ int[] primaryLowKeyFields = { 1 };
+ // second field from the tuples coming from secondary index
+ int[] primaryHighKeyFields = { 1 };
- // search primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBtreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories, true,
- primaryLowKeyFields, primaryHighKeyFields, true, true,
- opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBtreeSearchOp, NC1_ID);
+ // search primary index
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, primaryLowKeyFields,
+ primaryHighKeyFields, true, true, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
createTempFile().getAbsolutePath()) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- secondaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBtreeSearchOp, 0, printer, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryBtreeSearchOp, 0, primaryBtreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBtreeSearchOp, 0, printer, 0);
- spec.addRoot(printer);
- runTest(spec);
- }
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
- File secondary = new File(secondaryFileName);
- secondary.deleteOnExit();
- }
+ File secondary = new File(secondaryFileName);
+ secondary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 818dcd3..1e070a8 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -47,20 +46,20 @@
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
public class BTreeSecondaryIndexSearchOperatorTest extends
@@ -70,8 +69,8 @@
}
private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new BTreeOpHelperFactory();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
"ddMMyy-hhmmssSS");
@@ -136,17 +135,17 @@
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
- primaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ primaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
primaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
// field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ secondaryTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ secondaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
secondaryComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
secondaryComparatorFactories[1] = UTF8StringBinaryComparatorFactory.INSTANCE;
@@ -200,11 +199,11 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
+ spec, storageManager, indexRegistryProvider,
primaryBtreeSplitProvider, primaryInteriorFrameFactory,
primaryLeafFrameFactory, primaryTypeTraits,
primaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
+ dataflowHelperFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
primaryBtreeBulkLoad, NC1_ID);
@@ -246,10 +245,10 @@
// scan primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBtreeSplitProvider,
+ indexRegistryProvider, primaryBtreeSplitProvider,
primaryInteriorFrameFactory, primaryLeafFrameFactory,
primaryTypeTraits, primaryComparatorFactories, true,
- lowKeyFields, highKeyFields, true, true, opHelperFactory);
+ lowKeyFields, highKeyFields, true, true, dataflowHelperFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
primaryBtreeSearchOp, NC1_ID);
@@ -266,11 +265,11 @@
// load secondary index
int[] fieldPermutation = { 3, 0 };
TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
+ spec, storageManager, indexRegistryProvider,
secondaryBtreeSplitProvider, secondaryInteriorFrameFactory,
secondaryLeafFrameFactory, secondaryTypeTraits,
secondaryComparatorFactories, fieldPermutation, 0.7f,
- opHelperFactory);
+ dataflowHelperFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
secondaryBtreeBulkLoad, NC1_ID);
@@ -319,11 +318,11 @@
// search secondary index
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
spec, secondaryRecDesc, storageManager,
- treeIndexRegistryProvider, secondaryBtreeSplitProvider,
+ indexRegistryProvider, secondaryBtreeSplitProvider,
secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
secondaryTypeTraits, secondaryComparatorFactories, true,
secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- opHelperFactory);
+ dataflowHelperFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
secondaryBtreeSearchOp, NC1_ID);
@@ -335,11 +334,11 @@
// search primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(
spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBtreeSplitProvider,
+ indexRegistryProvider, primaryBtreeSplitProvider,
primaryInteriorFrameFactory, primaryLeafFrameFactory,
primaryTypeTraits, primaryComparatorFactories, true,
primaryLowKeyFields, primaryHighKeyFields, true, true,
- opHelperFactory);
+ dataflowHelperFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
primaryBtreeSearchOp, NC1_ID);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
new file mode 100644
index 0000000..76a60ae
--- /dev/null
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
@@ -0,0 +1,74 @@
+package edu.uci.ics.hyracks.tests.invertedindex;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+
+public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
+
+ @Test
+ public void tokenizerTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/cleanednumbereddblptitles.txt"))) };
+ IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
+ RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+ FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
+
+ RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+ ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+ IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+ tokenFactory);
+ int[] tokenFields = { 1 };
+ int[] keyFields = { 0 };
+ BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
+ tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
+
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+ createTempFile().getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizer, 0, printer, 0);
+
+ spec.addRoot(printer);
+ runTest(spec);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
new file mode 100644
index 0000000..6440db3
--- /dev/null
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -0,0 +1,315 @@
+/*
+ * 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.tests.invertedindex;
+
+import java.io.DataOutput;
+import java.io.File;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
+import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+
+public class WordInvertedIndexTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
+
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
+
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
+ private final String dateString = simpleDateFormat.format(new Date());
+ private final String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree" + dateString;
+ private final String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree" + dateString;
+ private final String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists" + dateString;
+
+ private IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
+ private IFileSplitProvider btreeFileSplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(
+ NC1_ID, new FileReference(new File(btreeFileName))) });
+ private IFileSplitProvider invListsFileSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(invListsFileName))) });
+
+ // Primary BTree index.
+ private int primaryFieldCount = 2;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private int primaryKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryTupleWriterFactory = new TypeAwareTupleWriterFactory(primaryTypeTraits);
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryLeafFrameFactory = new BTreeNSMLeafFrameFactory(primaryTupleWriterFactory);
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+ // Inverted index BTree dictionary.
+ private ITypeTrait[] tokenTypeTraits = new ITypeTrait[1];
+ private IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[1];
+
+ // Inverted index stuff.
+ private int invListElementFieldCount = 1;
+ private ITypeTrait[] invListsTypeTraits = new ITypeTrait[invListElementFieldCount];
+ private IBinaryComparatorFactory[] invListsComparatorFactories = new IBinaryComparatorFactory[invListElementFieldCount];
+ private RecordDescriptor tokenizerRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+ private RecordDescriptor invListsRecDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
+
+ // Tokenizer stuff.
+ private ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+ private IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+ tokenFactory);
+
+ @Before
+ public void setup() throws Exception {
+ // Field declarations and comparators for primary BTree index.
+ primaryTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+ // Field declarations and comparators for tokens.
+ tokenTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ tokenComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+
+ // Field declarations and comparators for inverted lists.
+ invListsTypeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ invListsComparatorFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+ loadPrimaryIndex();
+ printPrimaryIndex();
+ loadInvertedIndex();
+ }
+
+ @Test
+ public void testConjunctiveSearcher() throws Exception {
+ IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
+ searchInvertedIndex("of", conjunctiveSearchModifierFactory);
+ searchInvertedIndex("3d", conjunctiveSearchModifierFactory);
+ searchInvertedIndex("of the human", conjunctiveSearchModifierFactory);
+ }
+
+ private IOperatorDescriptor createFileScanOp(JobSpecification spec) {
+ FileSplit[] dblpTitleFileSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/cleanednumbereddblptitles.txt"))) };
+ IFileSplitProvider dblpTitleSplitProvider = new ConstantFileSplitProvider(dblpTitleFileSplits);
+ RecordDescriptor dblpTitleRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+ FileScanOperatorDescriptor dblpTitleScanner = new FileScanOperatorDescriptor(spec, dblpTitleSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE }, '|'), dblpTitleRecDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, dblpTitleScanner, NC1_ID);
+ return dblpTitleScanner;
+ }
+
+ private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
+ int[] fieldPermutation = { 0, 1 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ btreeDataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
+ return primaryBtreeBulkLoad;
+ }
+
+ private IOperatorDescriptor createScanKeyProviderOp(JobSpecification spec) throws HyracksDataException {
+ // build dummy tuple containing nothing
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
+ DataOutput dos = tb.getDataOutput();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+ tb.addFieldEndOffset();
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+ return keyProviderOp;
+ }
+
+ private IOperatorDescriptor createPrimaryScanOp(JobSpecification spec) throws HyracksDataException {
+ int[] lowKeyFields = null; // - infinity
+ int[] highKeyFields = null; // + infinity
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+ highKeyFields, true, true, btreeDataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+ return primaryBtreeSearchOp;
+ }
+
+ private void loadPrimaryIndex() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ // Assuming that the data is pre-sorted on the key. No need to sort
+ // before bulk load.
+ IOperatorDescriptor fileScanOp = createFileScanOp(spec);
+ IOperatorDescriptor primaryBulkLoad = createPrimaryBulkLoadOp(spec);
+ spec.connect(new OneToOneConnectorDescriptor(spec), fileScanOp, 0, primaryBulkLoad, 0);
+ spec.addRoot(primaryBulkLoad);
+ runTest(spec);
+ }
+
+ private void printPrimaryIndex() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
+ IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+ createTempFile().getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
+ spec.addRoot(printer);
+ runTest(spec);
+ }
+
+ private IOperatorDescriptor createExternalSortOp(JobSpecification spec, int[] sortFields,
+ RecordDescriptor outputRecDesc) {
+ ExternalSortOperatorDescriptor externalSortOp = new ExternalSortOperatorDescriptor(spec, 1000, sortFields,
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
+ IntegerBinaryComparatorFactory.INSTANCE }, outputRecDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, externalSortOp, NC1_ID);
+ return externalSortOp;
+ }
+
+ private IOperatorDescriptor createBinaryTokenizerOp(JobSpecification spec, int[] tokenFields, int[] keyFields) {
+ BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
+ tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
+ return binaryTokenizer;
+ }
+
+ private IOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec, int[] fieldPermutation) {
+ InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new InvertedIndexBulkLoadOperatorDescriptor(spec,
+ fieldPermutation, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
+ indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
+ invListsComparatorFactories, btreeDataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexBulkLoadOp, NC1_ID);
+ return invIndexBulkLoadOp;
+ }
+
+ public void loadInvertedIndex() throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
+ IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
+ int[] tokenFields = { 1 };
+ int[] keyFields = { 0 };
+ IOperatorDescriptor binaryTokenizerOp = createBinaryTokenizerOp(spec, tokenFields, keyFields);
+ int[] sortFields = { 0, 1 };
+ IOperatorDescriptor externalSortOp = createExternalSortOp(spec, sortFields, tokenizerRecDesc);
+ int[] fieldPermutation = { 0, 1 };
+ IOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec, fieldPermutation);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, binaryTokenizerOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), binaryTokenizerOp, 0, externalSortOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), externalSortOp, 0, invIndexBulkLoadOp, 0);
+ spec.addRoot(invIndexBulkLoadOp);
+ runTest(spec);
+ }
+
+ private IOperatorDescriptor createQueryProviderOp(JobSpecification spec, String queryString)
+ throws HyracksDataException {
+ // Build tuple with exactly one field, which is the query,
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+ DataOutput dos = tb.getDataOutput();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString, dos);
+ tb.addFieldEndOffset();
+ ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
+ ConstantTupleSourceOperatorDescriptor queryProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ queryRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, queryProviderOp, NC1_ID);
+ return queryProviderOp;
+ }
+
+ private IOperatorDescriptor createInvertedIndexSearchOp(JobSpecification spec,
+ IInvertedIndexSearchModifierFactory searchModifierFactory) {
+ InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(spec, 0,
+ storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
+ tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+ btreeDataflowHelperFactory, tokenizerFactory, searchModifierFactory, invListsRecDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexSearchOp, NC1_ID);
+ return invIndexSearchOp;
+ }
+
+ public void searchInvertedIndex(String queryString, IInvertedIndexSearchModifierFactory searchModifierFactory)
+ throws Exception {
+ JobSpecification spec = new JobSpecification();
+ IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
+ IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+ createTempFile().getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
+ spec.addRoot(printer);
+ runTest(spec);
+ }
+}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 4b31862..74dc4a8 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -49,193 +48,161 @@
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-public class RTreePrimaryIndexSearchOperatorTest extends
- AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+public class RTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new RTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new RTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary R-tree index
- private int primaryFieldCount = 5;
- private int primaryKeyFieldCount = 4;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ // field, type and key declarations for primary R-tree index
+ private int primaryFieldCount = 5;
+ private int primaryKeyFieldCount = 4;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- primaryTypeTraits);
+ private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
+ primaryTypeTraits);
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- private ITreeIndexFrameFactory primaryInteriorFrameFactory;
- private ITreeIndexFrameFactory primaryLeafFrameFactory;
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory;
+ private ITreeIndexFrameFactory primaryLeafFrameFactory;
- private static String primaryRTreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryRTreeName;
+ private static String primaryRTreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryRTreeName;
- private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary R-tree index
- primaryTypeTraits[0] = new TypeTrait(8);
- primaryTypeTraits[1] = new TypeTrait(8);
- primaryTypeTraits[2] = new TypeTrait(8);
- primaryTypeTraits[3] = new TypeTrait(8);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
- primaryComparatorFactories[1] = primaryComparatorFactories[0];
- primaryComparatorFactories[2] = primaryComparatorFactories[0];
- primaryComparatorFactories[3] = primaryComparatorFactories[0];
-
- IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
-
- primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory, primaryValueProviderFactories);
- primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory, primaryValueProviderFactories);
-
- loadPrimaryIndexTest();
- }
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary R-tree index
+ primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+ primaryComparatorFactories[1] = primaryComparatorFactories[0];
+ primaryComparatorFactories[2] = primaryComparatorFactories[0];
+ primaryComparatorFactories[3] = primaryComparatorFactories[0];
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
+ .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
- FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/spatial.txt"))) };
- IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(
- objectsSplits);
- RecordDescriptor objectsDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
+ primaryValueProviderFactories);
+ primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(primaryTupleWriterFactory, primaryValueProviderFactories);
- FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(
- spec, objectsSplitProvider,
- new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- objScanner, NC1_ID);
+ loadPrimaryIndexTest();
+ }
- int[] fieldPermutation = { 0, 1, 2, 3, 4 };
- TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryRTreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories,
- fieldPermutation, 0.7f, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryRTreeBulkLoad, NC1_ID);
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0,
- primaryRTreeBulkLoad, 0);
+ FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/spatial.txt"))) };
+ IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(objectsSplits);
+ RecordDescriptor objectsDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- spec.addRoot(primaryRTreeBulkLoad);
- runTest(spec);
- }
+ FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(spec, objectsSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { DoubleParserFactory.INSTANCE,
+ DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, objScanner, NC1_ID);
- @Test
- public void searchPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ int[] fieldPermutation = { 0, 1, 2, 3, 4 };
+ TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryRTreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeBulkLoad, NC1_ID);
- // build tuple
- ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount);
- DataOutput dos = tb.getDataOutput();
+ spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0, primaryRTreeBulkLoad, 0);
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
- tb.addFieldEndOffset();
+ spec.addRoot(primaryRTreeBulkLoad);
+ runTest(spec);
+ }
- ISerializerDeserializer[] keyRecDescSers = {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ @Test
+ public void searchPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ // build tuple
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount);
+ DataOutput dos = tb.getDataOutput();
- int[] keyFields = { 0, 1, 2, 3 };
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
+ tb.addFieldEndOffset();
- RTreeSearchOperatorDescriptor primaryRTreeSearchOp = new RTreeSearchOperatorDescriptor(
- spec, primaryRecDesc, storageManager,
- treeIndexRegistryProvider, primaryRTreeSplitProvider,
- primaryInteriorFrameFactory, primaryLeafFrameFactory,
- primaryTypeTraits, primaryComparatorFactories,
- keyFields, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryRTreeSearchOp, NC1_ID);
+ ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+ int[] keyFields = { 0, 1, 2, 3 };
+
+ RTreeSearchOperatorDescriptor primaryRTreeSearchOp = new RTreeSearchOperatorDescriptor(spec, primaryRecDesc,
+ storageManager, indexRegistryProvider, primaryRTreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, keyFields,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
createTempFile().getAbsolutePath()) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- primaryRTreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryRTreeSearchOp, 0, printer, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryRTreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryRTreeSearchOp, 0, printer, 0);
- spec.addRoot(printer);
- runTest(spec);
- }
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
- }
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index 1b1d760..62fc150 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -24,11 +24,11 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.DoubleBinaryComparatorFactory;
@@ -43,152 +43,135 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexStatsOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
public class RTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new RTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new RTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary R-tree index
- private int primaryFieldCount = 5;
- private int primaryKeyFieldCount = 4;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ // field, type and key declarations for primary R-tree index
+ private int primaryFieldCount = 5;
+ private int primaryKeyFieldCount = 4;
+ private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
+ private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
- private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- primaryTypeTraits);
+ private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
+ primaryTypeTraits);
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- private ITreeIndexFrameFactory primaryInteriorFrameFactory;
- private ITreeIndexFrameFactory primaryLeafFrameFactory;
+ private ITreeIndexFrameFactory primaryInteriorFrameFactory;
+ private ITreeIndexFrameFactory primaryLeafFrameFactory;
- private static String primaryRTreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryRTreeName;
+ private static String primaryRTreeName = "primary" + simpleDateFormat.format(new Date());
+ private static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + primaryRTreeName;
- private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary R-tree index
- primaryTypeTraits[0] = new TypeTrait(8);
- primaryTypeTraits[1] = new TypeTrait(8);
- primaryTypeTraits[2] = new TypeTrait(8);
- primaryTypeTraits[3] = new TypeTrait(8);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
- primaryComparatorFactories[1] = primaryComparatorFactories[0];
- primaryComparatorFactories[2] = primaryComparatorFactories[0];
- primaryComparatorFactories[3] = primaryComparatorFactories[0];
-
- IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
-
- primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory, primaryValueProviderFactories);
- primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory, primaryValueProviderFactories);
-
- loadPrimaryIndexTest();
- }
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary R-tree index
+ primaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+ primaryComparatorFactories[1] = primaryComparatorFactories[0];
+ primaryComparatorFactories[2] = primaryComparatorFactories[0];
+ primaryComparatorFactories[3] = primaryComparatorFactories[0];
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
+ .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
- FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/spatial.txt"))) };
- IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(
- objectsSplits);
- RecordDescriptor objectsDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(primaryTupleWriterFactory,
+ primaryValueProviderFactories);
+ primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(primaryTupleWriterFactory, primaryValueProviderFactories);
- FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(
- spec, objectsSplitProvider,
- new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- objScanner, NC1_ID);
+ loadPrimaryIndexTest();
+ }
- int[] fieldPermutation = { 0, 1, 2, 3, 4 };
- TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryRTreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories,
- fieldPermutation, 0.7f, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryRTreeBulkLoad, NC1_ID);
+ public void loadPrimaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0,
- primaryRTreeBulkLoad, 0);
+ FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/spatial.txt"))) };
+ IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(objectsSplits);
+ RecordDescriptor objectsDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- spec.addRoot(primaryRTreeBulkLoad);
- runTest(spec);
- }
+ FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(spec, objectsSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { DoubleParserFactory.INSTANCE,
+ DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, objScanner, NC1_ID);
- @Test
- public void showPrimaryIndexStats() throws Exception {
- JobSpecification spec = new JobSpecification();
+ int[] fieldPermutation = { 0, 1, 2, 3, 4 };
+ TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryRTreeSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeBulkLoad, NC1_ID);
- TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryRTreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryStatsOp, NC1_ID);
+ spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0, primaryRTreeBulkLoad, 0);
- spec.addRoot(primaryStatsOp);
- runTest(spec);
- }
+ spec.addRoot(primaryRTreeBulkLoad);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
- }
+ @Test
+ public void showPrimaryIndexStats() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
+ indexRegistryProvider, primaryRTreeSplitProvider, primaryInteriorFrameFactory, primaryLeafFrameFactory,
+ primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
+
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
+ createTempFile().getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
+ spec.addRoot(printer);
+ runTest(spec);
+ }
+
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primary = new File(primaryFileName);
+ primary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 5a24036..300a726 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -30,7 +30,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -50,18 +49,18 @@
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeOpHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
@@ -69,417 +68,257 @@
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.test.support.TestIndexRegistryProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
-import edu.uci.ics.hyracks.test.support.TestTreeIndexRegistryProvider;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-public class RTreeSecondaryIndexSearchOperatorTest extends
- AbstractIntegrationTest {
- static {
- TestStorageManagerComponentHolder.init(8192, 20, 20);
- }
+public class RTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
- private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
- private ITreeIndexOpHelperFactory opHelperFactory = new RTreeOpHelperFactory();
- private ITreeIndexOpHelperFactory bTreeopHelperFactory = new BTreeOpHelperFactory();
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<IIndex> indexRegistryProvider = new TestIndexRegistryProvider();
+ private IIndexDataflowHelperFactory dataflowHelperFactory = new RTreeDataflowHelperFactory();
+ private IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory();
- private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
- "ddMMyy-hhmmssSS");
- private final static String sep = System.getProperty("file.separator");
+ private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
+ private final static String sep = System.getProperty("file.separator");
- // field, type and key declarations for primary R-tree index
- private int primaryFieldCount = 5;
- private int primaryKeyFieldCount = 4;
- private ITypeTrait[] primaryTypeTraits = new ITypeTrait[primaryFieldCount];
- private IBinaryComparatorFactory[] primaryComparatorFactories = new IBinaryComparatorFactory[primaryKeyFieldCount];
+ // field, type and key declarations for primary B-tree index
+ private int primaryBTreeFieldCount = 10;
+ private ITypeTrait[] primaryBTreeTypeTraits = new ITypeTrait[primaryBTreeFieldCount];
+ private int primaryBTreeKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] primaryBTreeComparatorFactories = new IBinaryComparatorFactory[primaryBTreeKeyFieldCount];
+ private TypeAwareTupleWriterFactory primaryBTreeTupleWriterFactory = new TypeAwareTupleWriterFactory(
+ primaryBTreeTypeTraits);
+ private ITreeIndexFrameFactory primaryBTreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
+ primaryBTreeTupleWriterFactory);
+ private ITreeIndexFrameFactory primaryBTreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(
+ primaryBTreeTupleWriterFactory);
- private RTreeTypeAwareTupleWriterFactory primaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- primaryTypeTraits);
+ private static String primaryBTreeName = "primaryBTree" + simpleDateFormat.format(new Date());
+ private static String primaryBTreeFileName = System.getProperty("java.io.tmpdir") + sep + primaryBTreeName;
- private ITreeIndexFrameFactory primaryInteriorFrameFactory;
- private ITreeIndexFrameFactory primaryLeafFrameFactory;
+ private IFileSplitProvider primaryBTreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryBTreeFileName))) });
- private static String primaryRTreeName = "primary"
- + simpleDateFormat.format(new Date());
- private static String primaryFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryRTreeName;
+ private RecordDescriptor primaryBTreeRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE, });
- private IFileSplitProvider primaryRTreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryFileName))) });
+ // field, type and key declarations for secondary indexes
+ private int secondaryFieldCount = 5;
+ private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
+ private int secondaryKeyFieldCount = 4;
+ private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
+ private IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = new IPrimitiveValueProviderFactory[secondaryKeyFieldCount];
- private RecordDescriptor primaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ private RTreeTypeAwareTupleWriterFactory secondaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
+ secondaryTypeTraits);
- // field, type and key declarations for primary B-tree index
- private int primaryBTreeFieldCount = 10;
- private ITypeTrait[] primaryBTreeTypeTraits = new ITypeTrait[primaryBTreeFieldCount];
- private int primaryBTreeKeyFieldCount = 1;
- private IBinaryComparatorFactory[] primaryBTreeComparatorFactories = new IBinaryComparatorFactory[primaryBTreeKeyFieldCount];
- private TypeAwareTupleWriterFactory primaryBTreeTupleWriterFactory = new TypeAwareTupleWriterFactory(
- primaryBTreeTypeTraits);
- private ITreeIndexFrameFactory primaryBTreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(
- primaryBTreeTupleWriterFactory);
- private ITreeIndexFrameFactory primaryBTreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(
- primaryBTreeTupleWriterFactory);
+ private ITreeIndexFrameFactory secondaryInteriorFrameFactory;
+ private ITreeIndexFrameFactory secondaryLeafFrameFactory;
- private static String primaryBTreeName = "primaryBTree"
- + simpleDateFormat.format(new Date());
- private static String primaryBTreeFileName = System
- .getProperty("java.io.tmpdir") + sep + primaryBTreeName;
+ private static String secondaryRTreeName = "secondary" + simpleDateFormat.format(new Date());
+ private static String secondaryFileName = System.getProperty("java.io.tmpdir") + sep + secondaryRTreeName;
- private IFileSplitProvider primaryBTreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- primaryBTreeFileName))) });
+ private IFileSplitProvider secondaryRTreeSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(secondaryFileName))) });
- private RecordDescriptor primaryBTreeRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE, });
+ private RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
- // field, type and key declarations for secondary indexes
- private int secondaryFieldCount = 5;
- private ITypeTrait[] secondaryTypeTraits = new ITypeTrait[secondaryFieldCount];
- private int secondaryKeyFieldCount = 4;
- private IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[secondaryKeyFieldCount];
- private IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = new IPrimitiveValueProviderFactory[secondaryKeyFieldCount];
+ @Before
+ public void setup() throws Exception {
+ // field, type and key declarations for primary B-tree index
+ primaryBTreeTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryBTreeTypeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryBTreeTypeTraits[2] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryBTreeTypeTraits[3] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryBTreeTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryBTreeTypeTraits[5] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ primaryBTreeTypeTraits[6] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryBTreeTypeTraits[7] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryBTreeTypeTraits[8] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryBTreeTypeTraits[9] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ primaryBTreeComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- private RTreeTypeAwareTupleWriterFactory secondaryTupleWriterFactory = new RTreeTypeAwareTupleWriterFactory(
- secondaryTypeTraits);
+ // field, type and key declarations for secondary indexes
+ secondaryTypeTraits[0] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ secondaryTypeTraits[1] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ secondaryTypeTraits[2] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ secondaryTypeTraits[3] = ITypeTrait.DOUBLE_TYPE_TRAIT;
+ secondaryTypeTraits[4] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ secondaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
+ secondaryComparatorFactories[1] = secondaryComparatorFactories[0];
+ secondaryComparatorFactories[2] = secondaryComparatorFactories[0];
+ secondaryComparatorFactories[3] = secondaryComparatorFactories[0];
+ secondaryValueProviderFactories[0] = DoublePrimitiveValueProviderFactory.INSTANCE;
+ secondaryValueProviderFactories[1] = secondaryValueProviderFactories[0];
+ secondaryValueProviderFactories[2] = secondaryValueProviderFactories[0];
+ secondaryValueProviderFactories[3] = secondaryValueProviderFactories[0];
- private ITreeIndexFrameFactory secondaryInteriorFrameFactory;
- private ITreeIndexFrameFactory secondaryLeafFrameFactory;
+ IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = RTreeUtils
+ .comparatorFactoriesToPrimitiveValueProviderFactories(secondaryComparatorFactories);
- private static String secondaryRTreeName = "secondary"
- + simpleDateFormat.format(new Date());
- private static String secondaryFileName = System
- .getProperty("java.io.tmpdir") + sep + secondaryRTreeName;
+ secondaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(secondaryTupleWriterFactory,
+ secondaryValueProviderFactories);
+ secondaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(secondaryTupleWriterFactory,
+ secondaryValueProviderFactories);
- private IFileSplitProvider secondaryRTreeSplitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
- secondaryFileName))) });
+ loadPrimaryBTreeIndexTest();
+ loadSecondaryIndexTest();
+ }
- private RecordDescriptor secondaryRecDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ public void loadPrimaryBTreeIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- @Before
- public void setup() throws Exception {
- // field, type and key declarations for primary R-tree index
- primaryTypeTraits[0] = new TypeTrait(8);
- primaryTypeTraits[1] = new TypeTrait(8);
- primaryTypeTraits[2] = new TypeTrait(8);
- primaryTypeTraits[3] = new TypeTrait(8);
- primaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
- primaryComparatorFactories[1] = primaryComparatorFactories[0];
- primaryComparatorFactories[2] = primaryComparatorFactories[0];
- primaryComparatorFactories[3] = primaryComparatorFactories[0];
+ FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
+ "data/orders-with-locations.txt"))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE });
- IPrimitiveValueProviderFactory[] primaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(primaryComparatorFactories);
-
- primaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
- primaryTupleWriterFactory, primaryValueProviderFactories);
- primaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(
- primaryTupleWriterFactory, primaryValueProviderFactories);
-
- // field, type and key declarations for primary B-tree index
- primaryBTreeTypeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryBTreeTypeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryBTreeTypeTraits[2] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryBTreeTypeTraits[3] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryBTreeTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryBTreeTypeTraits[5] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- primaryBTreeTypeTraits[6] = new TypeTrait(8);
- primaryBTreeTypeTraits[7] = new TypeTrait(8);
- primaryBTreeTypeTraits[8] = new TypeTrait(8);
- primaryBTreeTypeTraits[9] = new TypeTrait(8);
- primaryBTreeComparatorFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE, DoubleParserFactory.INSTANCE,
+ DoubleParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- // field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = new TypeTrait(8);
- secondaryTypeTraits[1] = new TypeTrait(8);
- secondaryTypeTraits[2] = new TypeTrait(8);
- secondaryTypeTraits[3] = new TypeTrait(8);
- secondaryTypeTraits[4] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- secondaryComparatorFactories[0] = DoubleBinaryComparatorFactory.INSTANCE;
- secondaryComparatorFactories[1] = secondaryComparatorFactories[0];
- secondaryComparatorFactories[2] = secondaryComparatorFactories[0];
- secondaryComparatorFactories[3] = secondaryComparatorFactories[0];
- secondaryValueProviderFactories[0] = DoublePrimitiveValueProviderFactory.INSTANCE;
- secondaryValueProviderFactories[1] = secondaryValueProviderFactories[0];
- secondaryValueProviderFactories[2] = secondaryValueProviderFactories[0];
- secondaryValueProviderFactories[3] = secondaryValueProviderFactories[0];
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
+ new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = RTreeUtils
- .comparatorFactoriesToPrimitiveValueProviderFactories(secondaryComparatorFactories);
-
- secondaryInteriorFrameFactory = new RTreeNSMInteriorFrameFactory(
- secondaryTupleWriterFactory, secondaryValueProviderFactories);
- secondaryLeafFrameFactory = new RTreeNSMLeafFrameFactory(
- secondaryTupleWriterFactory, secondaryValueProviderFactories);
-
- loadPrimaryIndexTest();
- loadPrimaryBTreeIndexTest();
- loadSecondaryIndexTest();
- }
+ int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
+ TreeIndexBulkLoadOperatorDescriptor primaryBTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, primaryBTreeSplitProvider, primaryBTreeInteriorFrameFactory,
+ primaryBTreeLeafFrameFactory, primaryBTreeTypeTraits, primaryBTreeComparatorFactories,
+ fieldPermutation, 0.7f, btreeDataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBTreeBulkLoad, NC1_ID);
- public void loadPrimaryBTreeIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/orders-with-locations.txt"))) };
- IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(
- ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE });
+ spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, primaryBTreeBulkLoad, 0);
- FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(
- spec, ordersSplitProvider, new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] {
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE }, '|'),
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- ordScanner, NC1_ID);
+ spec.addRoot(primaryBTreeBulkLoad);
+ runTest(spec);
+ }
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
- spec,
- 1000,
- new int[] { 0 },
- new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE },
- ordersDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter,
- NC1_ID);
+ public void loadSecondaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
- TreeIndexBulkLoadOperatorDescriptor primaryBTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryBTreeSplitProvider, primaryBTreeInteriorFrameFactory,
- primaryBTreeLeafFrameFactory, primaryBTreeTypeTraits,
- primaryBTreeComparatorFactories, fieldPermutation, 0.7f,
- bTreeopHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBTreeBulkLoad, NC1_ID);
+ // build dummy tuple containing nothing
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryBTreeKeyFieldCount * 2);
+ DataOutput dos = tb.getDataOutput();
- spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0,
- sorter, 0);
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
+ tb.addFieldEndOffset();
- spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0,
- primaryBTreeBulkLoad, 0);
+ ISerializerDeserializer[] keyRecDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- spec.addRoot(primaryBTreeBulkLoad);
- runTest(spec);
- }
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- public void loadPrimaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ int[] lowKeyFields = null; // - infinity
+ int[] highKeyFields = null; // + infinity
- FileSplit[] objectsSplits = new FileSplit[] { new FileSplit(NC1_ID,
- new FileReference(new File("data/spatial.txt"))) };
- IFileSplitProvider objectsSplitProvider = new ConstantFileSplitProvider(
- objectsSplits);
- RecordDescriptor objectsDesc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ // scan primary index
+ BTreeSearchOperatorDescriptor primaryBTreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+ primaryBTreeRecDesc, storageManager, indexRegistryProvider, primaryBTreeSplitProvider,
+ primaryBTreeInteriorFrameFactory, primaryBTreeLeafFrameFactory, primaryBTreeTypeTraits,
+ primaryBTreeComparatorFactories, true, lowKeyFields, highKeyFields, true, true,
+ btreeDataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBTreeSearchOp, NC1_ID);
- FileScanOperatorDescriptor objScanner = new FileScanOperatorDescriptor(
- spec, objectsSplitProvider,
- new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- DoubleParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'), objectsDesc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- objScanner, NC1_ID);
+ // load secondary index
+ int[] fieldPermutation = { 6, 7, 8, 9, 0 };
+ TreeIndexBulkLoadOperatorDescriptor secondaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, secondaryRTreeSplitProvider, secondaryInteriorFrameFactory,
+ secondaryLeafFrameFactory, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
+ dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeBulkLoad, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 3, 4 };
- TreeIndexBulkLoadOperatorDescriptor primaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- primaryRTreeSplitProvider, primaryInteriorFrameFactory,
- primaryLeafFrameFactory, primaryTypeTraits,
- primaryComparatorFactories,
- fieldPermutation, 0.7f, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryRTreeBulkLoad, NC1_ID);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBTreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), primaryBTreeSearchOp, 0, secondaryRTreeBulkLoad, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec), objScanner, 0,
- primaryRTreeBulkLoad, 0);
+ spec.addRoot(secondaryRTreeBulkLoad);
+ runTest(spec);
+ }
- spec.addRoot(primaryRTreeBulkLoad);
- runTest(spec);
- }
+ @Test
+ public void searchSecondaryIndexTest() throws Exception {
+ JobSpecification spec = new JobSpecification();
- public void loadSecondaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
+ // build tuple
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
+ DataOutput dos = tb.getDataOutput();
- // build dummy tuple containing nothing
- ArrayTupleBuilder tb = new ArrayTupleBuilder(primaryKeyFieldCount * 2);
- DataOutput dos = tb.getDataOutput();
+ tb.reset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
+ tb.addFieldEndOffset();
+ DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
+ tb.addFieldEndOffset();
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize("0", dos);
- tb.addFieldEndOffset();
+ ISerializerDeserializer[] keyRecDescSers = { DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
- ISerializerDeserializer[] keyRecDescSers = {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
+ keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
+ int[] keyFields = { 0, 1, 2, 3 };
- int[] lowKeyFields = null; // - infinity
- int[] highKeyFields = null; // + infinity
-
- // scan primary index
- BTreeSearchOperatorDescriptor primaryBTreeSearchOp = new BTreeSearchOperatorDescriptor(
- spec, primaryBTreeRecDesc, storageManager,
- treeIndexRegistryProvider, primaryBTreeSplitProvider,
- primaryBTreeInteriorFrameFactory, primaryBTreeLeafFrameFactory,
- primaryBTreeTypeTraits, primaryBTreeComparatorFactories, true,
- lowKeyFields, highKeyFields, true, true, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- primaryBTreeSearchOp, NC1_ID);
-
- // load secondary index
- int[] fieldPermutation = { 6, 7, 8, 9, 0 };
- TreeIndexBulkLoadOperatorDescriptor secondaryRTreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(
- spec, storageManager, treeIndexRegistryProvider,
- secondaryRTreeSplitProvider, secondaryInteriorFrameFactory,
- secondaryLeafFrameFactory, secondaryTypeTraits,
- secondaryComparatorFactories,
- fieldPermutation, 0.7f, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryRTreeBulkLoad, NC1_ID);
-
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- primaryBTreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- primaryBTreeSearchOp, 0, secondaryRTreeBulkLoad, 0);
-
- spec.addRoot(secondaryRTreeBulkLoad);
- runTest(spec);
- }
-
- @Test
- public void searchSecondaryIndexTest() throws Exception {
- JobSpecification spec = new JobSpecification();
-
- // build tuple
- ArrayTupleBuilder tb = new ArrayTupleBuilder(secondaryKeyFieldCount);
- DataOutput dos = tb.getDataOutput();
-
- tb.reset();
- DoubleSerializerDeserializer.INSTANCE.serialize(61.2894, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(-149.624, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(61.8894, dos);
- tb.addFieldEndOffset();
- DoubleSerializerDeserializer.INSTANCE.serialize(-149.024, dos);
- tb.addFieldEndOffset();
-
- ISerializerDeserializer[] keyRecDescSers = {
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE,
- DoubleSerializerDeserializer.INSTANCE };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(
- spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
- tb.getSize());
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- keyProviderOp, NC1_ID);
-
- int[] keyFields = { 0, 1, 2, 3 };
-
- RTreeSearchOperatorDescriptor secondaryRTreeSearchOp = new RTreeSearchOperatorDescriptor(
- spec, secondaryRecDesc, storageManager,
- treeIndexRegistryProvider, secondaryRTreeSplitProvider,
- secondaryInteriorFrameFactory, secondaryLeafFrameFactory,
- secondaryTypeTraits, secondaryComparatorFactories,
- keyFields, opHelperFactory);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- secondaryRTreeSearchOp, NC1_ID);
+ RTreeSearchOperatorDescriptor secondaryRTreeSearchOp = new RTreeSearchOperatorDescriptor(spec,
+ secondaryRecDesc, storageManager, indexRegistryProvider, secondaryRTreeSplitProvider,
+ secondaryInteriorFrameFactory, secondaryLeafFrameFactory, secondaryTypeTraits,
+ secondaryComparatorFactories, keyFields, dataflowHelperFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
createTempFile().getAbsolutePath()) });
IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0,
- secondaryRTreeSearchOp, 0);
- spec.connect(new OneToOneConnectorDescriptor(spec),
- secondaryRTreeSearchOp, 0, printer, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryRTreeSearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondaryRTreeSearchOp, 0, printer, 0);
- spec.addRoot(printer);
- runTest(spec);
- }
+ spec.addRoot(printer);
+ runTest(spec);
+ }
- @AfterClass
- public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- primary.deleteOnExit();
+ @AfterClass
+ public static void cleanup() throws Exception {
+ File primaryBTree = new File(primaryBTreeFileName);
+ primaryBTree.deleteOnExit();
- File primaryBTree = new File(primaryBTreeFileName);
- primaryBTree.deleteOnExit();
-
- File secondary = new File(secondaryFileName);
- secondary.deleteOnExit();
- }
+ File secondary = new File(secondaryFileName);
+ secondary.deleteOnExit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
index 3f7bc30..8e88c18 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/compressors/FieldPrefixCompressor.java
@@ -178,14 +178,14 @@
// number of fields we decided to use for compression of
// this keyPartition
- int numFieldsToCompress = keyPartitions.get(kpIndex).maxPmiIndex + 1;
+ int fieldCountToCompress = keyPartitions.get(kpIndex).maxPmiIndex + 1;
int segmentStart = keyPartitions.get(kpIndex).firstTupleIndex;
int tuplesInSegment = 1;
// System.out.println("PROCESSING KEYPARTITION: " + kpIndex
// + " RANGE: " + keyPartitions.get(kpIndex).firstRecSlotNum
// + " " + keyPartitions.get(kpIndex).lastRecSlotNum +
- // " FIELDSTOCOMPRESS: " + numFieldsToCompress);
+ // " FIELDSTOCOMPRESS: " + fieldCountToCompress);
FieldPrefixTupleReference prevTuple = new FieldPrefixTupleReference(tupleWriter
.createTupleReference());
@@ -198,10 +198,10 @@
prevTuple.resetByTupleIndex(frame, i - 1);
tuple.resetByTupleIndex(frame, i);
- // check if tuples match in numFieldsToCompress of their
+ // check if tuples match in fieldCountToCompress of their
// first fields
int prefixFieldsMatch = 0;
- for (int j = 0; j < numFieldsToCompress; j++) {
+ for (int j = 0; j < fieldCountToCompress; j++) {
if (cmps[j].compare(pageArray, prevTuple.getFieldStart(j), prevTuple.getFieldLength(j),
pageArray, tuple.getFieldStart(j), tuple.getFieldLength(j)) == 0)
prefixFieldsMatch++;
@@ -212,7 +212,7 @@
// the two tuples must match in exactly the number of
// fields we decided to compress for this keyPartition
int processSegments = 0;
- if (prefixFieldsMatch == numFieldsToCompress)
+ if (prefixFieldsMatch == fieldCountToCompress)
tuplesInSegment++;
else
processSegments++;
@@ -223,7 +223,7 @@
for (int r = 0; r < processSegments; r++) {
// compress current segment and then start new
// segment
- if (tuplesInSegment < occurrenceThreshold || numFieldsToCompress <= 0) {
+ if (tuplesInSegment < occurrenceThreshold || fieldCountToCompress <= 0) {
// segment does not have at least
// occurrenceThreshold tuples, so write tuples
// uncompressed
@@ -240,19 +240,19 @@
// extract prefix, write prefix tuple to buffer,
// and set prefix slot
newPrefixSlots[newPrefixSlots.length - 1 - prefixTupleIndex] = slotManager
- .encodeSlotFields(numFieldsToCompress, prefixFreeSpace);
+ .encodeSlotFields(fieldCountToCompress, prefixFreeSpace);
// int tmp = freeSpace;
// prevRec.reset();
// System.out.println("SOURCE CONTENTS: " +
// buf.getInt(prevRec.getFieldOff()) + " " +
// buf.getInt(prevRec.getFieldOff()+4));
- prefixFreeSpace += tupleWriter.writeTupleFields(prevTuple, 0, numFieldsToCompress,
+ prefixFreeSpace += tupleWriter.writeTupleFields(prevTuple, 0, fieldCountToCompress,
byteBuffer, prefixFreeSpace);
// System.out.println("WRITING PREFIX RECORD " +
// prefixSlotNum + " AT " + tmp + " " +
// freeSpace);
// System.out.print("CONTENTS: ");
- // for(int x = 0; x < numFieldsToCompress; x++)
+ // for(int x = 0; x < fieldCountToCompress; x++)
// System.out.print(buf.getInt(tmp + x*4) +
// " ");
// System.out.println();
@@ -264,8 +264,8 @@
tupleToWrite.resetByTupleIndex(frame, currTupleIndex);
newTupleSlots[tupleCount - 1 - currTupleIndex] = slotManager.encodeSlotFields(
prefixTupleIndex, tupleFreeSpace);
- tupleFreeSpace += tupleWriter.writeTupleFields(tupleToWrite, numFieldsToCompress,
- fieldCount - numFieldsToCompress, byteBuffer, tupleFreeSpace);
+ tupleFreeSpace += tupleWriter.writeTupleFields(tupleToWrite, fieldCountToCompress,
+ fieldCount - fieldCountToCompress, byteBuffer, tupleFreeSpace);
}
prefixTupleIndex++;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java
new file mode 100644
index 0000000..3e235b3
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelper.java
@@ -0,0 +1,48 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class BTreeDataflowHelper extends TreeIndexDataflowHelper {
+ public BTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+ boolean createIfNotExists) {
+ super(opDesc, ctx, partition, createIfNotExists);
+ }
+
+ @Override
+ public ITreeIndex createIndexInstance() throws HyracksDataException {
+ MultiComparator cmp = IndexUtils.createMultiComparator(treeOpDesc.getTreeIndexComparatorFactories());
+ IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+ ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0,
+ metaDataFrameFactory);
+ return new BTree(bufferCache, treeOpDesc.getTreeIndexTypeTraits().length, cmp, freePageManager,
+ treeOpDesc.getTreeIndexInteriorFactory(), treeOpDesc.getTreeIndexLeafFactory());
+ }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelperFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelperFactory.java
new file mode 100644
index 0000000..308e305
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDataflowHelperFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+
+public class BTreeDataflowHelperFactory implements IIndexDataflowHelperFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, boolean createIfNotExists) {
+ return new BTreeDataflowHelper(opDesc, ctx, partition, createIfNotExists);
+ }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
deleted file mode 100644
index 09a809b..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
+++ /dev/null
@@ -1,34 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-
-public class BTreeOpHelper extends TreeIndexOpHelper {
-
- protected ITreeIndexOperatorDescriptorHelper opDesc;
-
- public BTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksTaskContext ctx, int partition,
- IndexHelperOpenMode mode) {
- super(opDesc, ctx, partition, mode);
- this.opDesc = opDesc;
- }
-
- public ITreeIndex createTreeIndex() throws HyracksDataException {
- IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0,
- metaDataFrameFactory);
- return new BTree(bufferCache, opDesc.getTreeIndexFieldCount(), cmp, freePageManager, opDesc.getTreeIndexInteriorFactory(),
- opDesc.getTreeIndexLeafFactory());
- }
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
deleted file mode 100644
index 6cc79ab..0000000
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-
-public class BTreeOpHelperFactory implements ITreeIndexOpHelperFactory {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
- IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode) {
- return new BTreeOpHelper(opDesc, ctx, partition, mode);
- }
-
-}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 649cb3f..4075152 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -23,32 +23,32 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public class BTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
private static final long serialVersionUID = 1L;
- private boolean isForward;
- private int[] lowKeyFields; // fields in input tuple to be used as low keys
- private int[] highKeyFields; // fields in input tuple to be used as high
+ protected boolean isForward;
+ protected int[] lowKeyFields; // fields in input tuple to be used as low keys
+ protected int[] highKeyFields; // fields in input tuple to be used as high
// keys
- private boolean lowKeyInclusive;
- private boolean highKeyInclusive;
+ protected boolean lowKeyInclusive;
+ protected boolean highKeyInclusive;
public BTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
- IStorageManagerInterface storageManager, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+ IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
- boolean lowKeyInclusive, boolean highKeyInclusive, ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider, fileSplitProvider, interiorFrameFactory,
- leafFrameFactory, typeTraits, comparatorFactories, opHelperFactory);
+ boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory) {
+ super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+ leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
this.isForward = isForward;
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
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 8cfd5d6..8a8c5ce 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
@@ -18,7 +18,6 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -30,47 +29,46 @@
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
public class BTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private TreeIndexOpHelper treeIndexOpHelper;
- private FrameTupleAccessor accessor;
+ protected TreeIndexDataflowHelper treeIndexHelper;
+ protected FrameTupleAccessor accessor;
- private ByteBuffer writeBuffer;
- private FrameTupleAppender appender;
- private ArrayTupleBuilder tb;
- private DataOutput dos;
+ protected ByteBuffer writeBuffer;
+ protected FrameTupleAppender appender;
+ protected ArrayTupleBuilder tb;
+ protected DataOutput dos;
- private BTree btree;
- private boolean isForward;
- private PermutingFrameTupleReference lowKey;
- private PermutingFrameTupleReference highKey;
- private boolean lowKeyInclusive;
- private boolean highKeyInclusive;
- private RangePredicate rangePred;
- private MultiComparator lowKeySearchCmp;
- private MultiComparator highKeySearchCmp;
- private ITreeIndexCursor cursor;
- private ITreeIndexFrame cursorFrame;
- private BTreeOpContext opCtx;
+ protected BTree btree;
+ protected boolean isForward;
+ protected PermutingFrameTupleReference lowKey;
+ protected PermutingFrameTupleReference highKey;
+ protected boolean lowKeyInclusive;
+ protected boolean highKeyInclusive;
+ protected RangePredicate rangePred;
+ protected MultiComparator lowKeySearchCmp;
+ protected MultiComparator highKeySearchCmp;
+ protected ITreeIndexCursor cursor;
+ protected ITreeIndexFrame cursorFrame;
+ protected ITreeIndexAccessor indexAccessor;
- private RecordDescriptor recDesc;
+ protected RecordDescriptor recDesc;
public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.OPEN);
+ treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ opDesc, ctx, partition, false);
this.isForward = isForward;
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
@@ -87,70 +85,48 @@
@Override
public void open() throws HyracksDataException {
- AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
+ AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
.getOperatorDescriptor();
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
cursorFrame = opDesc.getTreeIndexLeafFactory().createFrame();
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame);
+ setCursor();
writer.open();
try {
- treeIndexOpHelper.init();
- btree = (BTree) treeIndexOpHelper.getTreeIndex();
+ treeIndexHelper.init();
+ btree = (BTree) treeIndexHelper.getIndex();
- // construct range predicate
-
- // TODO: Can we construct the multicmps using helper methods?
- int lowKeySearchFields = btree.getMultiComparator().getComparators().length;
- int highKeySearchFields = btree.getMultiComparator().getComparators().length;
- if (lowKey != null)
- lowKeySearchFields = lowKey.getFieldCount();
- if (highKey != null)
- highKeySearchFields = highKey.getFieldCount();
-
- IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
- for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getMultiComparator().getComparators()[i];
- }
- lowKeySearchCmp = new MultiComparator(lowKeySearchComparators);
-
- if (lowKeySearchFields == highKeySearchFields) {
- highKeySearchCmp = lowKeySearchCmp;
- } else {
- IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
- for (int i = 0; i < highKeySearchFields; i++) {
- highKeySearchComparators[i] = btree.getMultiComparator().getComparators()[i];
- }
- highKeySearchCmp = new MultiComparator(highKeySearchComparators);
-
- }
-
+ // Construct range predicate.
+ lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getMultiComparator(), lowKey);
+ highKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getMultiComparator(), highKey);
rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
highKeySearchCmp);
- writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+ writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
tb = new ArrayTupleBuilder(btree.getFieldCount());
dos = tb.getDataOutput();
- appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+ appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
-
- opCtx = btree.createOpContext(IndexOp.SEARCH);
-
+ indexAccessor = btree.createAccessor();
} catch (Exception e) {
- treeIndexOpHelper.deinit();
+ treeIndexHelper.deinit();
throw new HyracksDataException(e);
}
}
- private void writeSearchResults() throws Exception {
+ protected void setCursor() {
+ cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, false);
+ }
+
+ protected void writeSearchResults() throws Exception {
while (cursor.hasNext()) {
tb.reset();
cursor.next();
- ITupleReference frameTuple = cursor.getTuple();
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+ ITupleReference tuple = cursor.getTuple();
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
tb.addFieldEndOffset();
}
@@ -167,19 +143,19 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
accessor.reset(buffer);
-
int tupleCount = accessor.getTupleCount();
try {
for (int i = 0; i < tupleCount; i++) {
- if (lowKey != null)
+ if (lowKey != null) {
lowKey.reset(accessor, i);
- if (highKey != null)
+ }
+ if (highKey != null) {
highKey.reset(accessor, i);
+ }
rangePred.setLowKey(lowKey, lowKeyInclusive);
rangePred.setHighKey(highKey, highKeyInclusive);
-
cursor.reset();
- btree.search(cursor, rangePred, opCtx);
+ indexAccessor.search(cursor, rangePred);
writeSearchResults();
}
} catch (Exception e) {
@@ -200,7 +176,7 @@
throw new HyracksDataException(e);
}
} finally {
- treeIndexOpHelper.deinit();
+ treeIndexHelper.deinit();
}
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
new file mode 100644
index 0000000..2be1641
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleUpdaterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class BTreeUpdateSearchOperatorDescriptor extends BTreeSearchOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ITupleUpdaterFactory tupleUpdaterFactory;
+
+ public BTreeUpdateSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+ IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+ IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, boolean isForward, int[] lowKeyFields, int[] highKeyFields,
+ boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
+ ITupleUpdaterFactory tupleUpdaterFactory) {
+ super(spec, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+ leafFrameFactory, typeTraits, comparatorFactories, isForward, lowKeyFields, highKeyFields, lowKeyInclusive,
+ highKeyInclusive, dataflowHelperFactory);
+ this.tupleUpdaterFactory = tupleUpdaterFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
+ int partition, int nPartitions) {
+ return new BTreeUpdateSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive, tupleUpdaterFactory.createTupleUpdater());
+ }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
new file mode 100644
index 0000000..06a42bd
--- /dev/null
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.hyracks.storage.am.btree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITupleUpdater;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+
+public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNodePushable {
+ private final ITupleUpdater tupleUpdater;
+
+ public BTreeUpdateSearchOperatorNodePushable(
+ AbstractTreeIndexOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition,
+ IRecordDescriptorProvider recordDescProvider, boolean isForward,
+ int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
+ boolean highKeyInclusive, ITupleUpdater tupleUpdater) {
+ super(opDesc, ctx, partition, recordDescProvider, isForward, lowKeyFields,
+ highKeyFields, lowKeyInclusive, highKeyInclusive);
+ this.tupleUpdater = tupleUpdater;
+ }
+
+ @Override
+ protected void setCursor() {
+ cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame, true);
+ }
+
+ @Override
+ protected void writeSearchResults() throws Exception {
+ while (cursor.hasNext()) {
+ tb.reset();
+ cursor.next();
+ ITupleReference tuple = cursor.getTuple();
+ tupleUpdater.updateTuple(tuple);
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ appender.reset(writeBuffer, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ }
+}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java
index 1e09658..c568f6f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/exceptions/BTreeException.java
@@ -20,7 +20,6 @@
public class BTreeException extends TreeIndexException {
protected static final long serialVersionUID = 1L;
- protected boolean handled = false;
public BTreeException(Exception e) {
super(e);
@@ -29,12 +28,4 @@
public BTreeException(String message) {
super(message);
}
-
- public void setHandled(boolean handled) {
- this.handled = handled;
- }
-
- public boolean getHandled() {
- return handled;
- }
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
index 0334a7f..56e0774 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeFieldPrefixNSMLeafFrame.java
@@ -268,11 +268,11 @@
int newTupleBytes = 0;
int numPrefixFields = frameTuple.getNumPrefixFields();
- int numFields = frameTuple.getFieldCount();
+ int fieldCount = frameTuple.getFieldCount();
if (numPrefixFields != 0) {
// Check the space requirements for updating the suffix of the original tuple.
oldTupleBytes = frameTuple.getSuffixTupleSize();
- newTupleBytes = tupleWriter.bytesRequired(newTuple, numPrefixFields, numFields - numPrefixFields);
+ newTupleBytes = tupleWriter.bytesRequired(newTuple, numPrefixFields, fieldCount - numPrefixFields);
} else {
// The original tuple is uncompressed.
oldTupleBytes = frameTuple.getTupleSize();
@@ -308,18 +308,18 @@
int suffixTupleStartOff = slotManager.decodeSecondSlotField(tupleSlot);
frameTuple.resetByTupleIndex(this, tupleIndex);
- int numFields = frameTuple.getFieldCount();
+ int fieldCount = frameTuple.getFieldCount();
int numPrefixFields = frameTuple.getNumPrefixFields();
int oldTupleBytes = frameTuple.getSuffixTupleSize();
int bytesWritten = 0;
if (inPlace) {
// Overwrite the old tuple suffix in place.
- bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, numFields - numPrefixFields, buf, suffixTupleStartOff);
+ bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields, buf, suffixTupleStartOff);
} else {
// Insert the new tuple suffix at the end of the free space, and change the slot value (effectively "deleting" the old tuple).
int newSuffixTupleStartOff = buf.getInt(freeSpaceOff);
- bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, numFields - numPrefixFields, buf, newSuffixTupleStartOff);
+ bytesWritten = tupleWriter.writeTupleFields(newTuple, numPrefixFields, fieldCount - numPrefixFields, buf, newSuffixTupleStartOff);
// Update slot value using the same prefix slot num.
slotManager.setSlot(tupleSlotOff, slotManager.encodeSlotFields(prefixSlotNum, newSuffixTupleStartOff));
// Update contiguous free space pointer.
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 3952f5d..57f1b79 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
@@ -30,14 +30,16 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrame;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
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.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
@@ -118,10 +120,8 @@
ctx.freePages.clear();
}
- @Override
- public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException {
+ private void diskOrderScan(ITreeIndexCursor icursor, BTreeOpContext ctx) throws HyracksDataException {
TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
- BTreeOpContext ctx = (BTreeOpContext) ictx;
ctx.reset();
int currentPageId = rootPage;
@@ -143,9 +143,10 @@
}
}
- public void search(ITreeIndexCursor cursor, RangePredicate pred, BTreeOpContext ctx) throws Exception {
+ private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, BTreeOpContext ctx)
+ throws TreeIndexException, HyracksDataException, PageAllocationException {
ctx.reset();
- ctx.pred = pred;
+ ctx.pred = (RangePredicate) searchPred;
ctx.cursor = cursor;
// simple index scan
if (ctx.pred.getLowKeyComparator() == null) {
@@ -204,7 +205,7 @@
}
}
- private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
// Make sure the root is always in the same page.
ICachedPage leftNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, ctx.splitKey.getLeftPage()),
false);
@@ -250,8 +251,7 @@
}
}
- private void insertUpdateOrDelete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- BTreeOpContext ctx = (BTreeOpContext) ictx;
+ private void insertUpdateOrDelete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
ctx.reset();
ctx.pred.setLowKeyComparator(cmp);
ctx.pred.setHighKeyComparator(cmp);
@@ -287,25 +287,22 @@
}
}
- @Override
- public void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- insertUpdateOrDelete(tuple, ictx);
+ private void insert(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ insertUpdateOrDelete(tuple, ctx);
}
- @Override
- public void update(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+ private void update(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
// This call only allows updating of non-key fields.
// Updating a tuple's key necessitates deleting the old entry, and inserting the new entry.
// The user of the BTree is responsible for dealing with non-key updates (i.e., doing a delete + insert).
if (fieldCount == cmp.getKeyFieldCount()) {
throw new BTreeNotUpdateableException("Cannot perform updates when the entire tuple forms the key.");
}
- insertUpdateOrDelete(tuple, ictx);
+ insertUpdateOrDelete(tuple, ctx);
}
- @Override
- public void delete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- insertUpdateOrDelete(tuple, ictx);
+ private void delete(ITupleReference tuple, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ insertUpdateOrDelete(tuple, ctx);
}
private void insertLeaf(ICachedPage node, int pageId, ITupleReference tuple, BTreeOpContext ctx) throws Exception {
@@ -363,7 +360,7 @@
true);
rightNode.acquireWriteLatch();
try {
- IBTreeLeafFrame rightFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
+ IBTreeLeafFrame rightFrame = ctx.createLeafFrame();
rightFrame.setPage(rightNode);
rightFrame.initBuffer((byte) 0);
rightFrame.setMultiComparator(cmp);
@@ -466,7 +463,7 @@
ICachedPage rightNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, rightPageId), true);
rightNode.acquireWriteLatch();
try {
- IBTreeFrame rightFrame = (IBTreeFrame)interiorFrameFactory.createFrame();
+ IBTreeFrame rightFrame = ctx.createInteriorFrame();
rightFrame.setPage(rightNode);
rightFrame.initBuffer((byte) ctx.interiorFrame.getLevel());
rightFrame.setMultiComparator(cmp);
@@ -523,7 +520,7 @@
}
// Leaf will become empty.
- IBTreeLeafFrame siblingFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ IBTreeLeafFrame siblingFrame = ctx.createLeafFrame();
siblingFrame.setMultiComparator(cmp);
ICachedPage leftNode = null;
ICachedPage rightNode = null;
@@ -631,19 +628,19 @@
}
}
- private final void acquireLatch(ICachedPage node, IndexOp op, boolean isLeaf) {
- if (isLeaf && (op == IndexOp.INSERT || op == IndexOp.DELETE || op == IndexOp.UPDATE)) {
- node.acquireWriteLatch();
- } else {
+ private final void acquireLatch(ICachedPage node, BTreeOpContext ctx, boolean isLeaf) {
+ if (!isLeaf || (ctx.op == IndexOp.SEARCH && !ctx.cursor.exclusiveLatchNodes())) {
node.acquireReadLatch();
+ } else {
+ node.acquireWriteLatch();
}
}
- private final void releaseLatch(ICachedPage node, IndexOp op, boolean isLeaf) {
- if (isLeaf && (op == IndexOp.INSERT || op == IndexOp.DELETE || op == IndexOp.UPDATE)) {
- node.releaseWriteLatch();
- } else {
+ private final void releaseLatch(ICachedPage node, BTreeOpContext ctx, boolean isLeaf) {
+ if (!isLeaf || (ctx.op == IndexOp.SEARCH && !ctx.cursor.exclusiveLatchNodes())) {
node.releaseReadLatch();
+ } else {
+ node.releaseWriteLatch();
}
}
@@ -661,19 +658,14 @@
return isConsistent;
}
- private void performOp(int pageId, ICachedPage parent, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ private void performOp(int pageId, ICachedPage parent, BTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
ctx.interiorFrame.setPage(node);
- //System.out.println("PAGEID: " + pageId);
- //System.out.println("NODE: " + node);
- //System.out.println("LEVEL: " + ctx.interiorFrame.getLevel() + " " + ctx.interiorFrame.isLeaf());
- //System.out.println("-------------------------");
-
// this check performs an unprotected read in the page
// the following could happen: TODO fill out
boolean unsafeIsLeaf = ctx.interiorFrame.isLeaf();
- acquireLatch(node, ctx.op, unsafeIsLeaf);
+ acquireLatch(node, ctx, unsafeIsLeaf);
boolean smFlag = ctx.interiorFrame.getSmFlag();
// re-check leafness after latching
boolean isLeaf = ctx.interiorFrame.isLeaf();
@@ -752,7 +744,7 @@
ctx.opRestarts++;
System.out.println("ONGOING SM ON PAGE " + pageId + " AT LEVEL " + ctx.interiorFrame.getLevel()
+ ", RESTARTS: " + ctx.opRestarts);
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
// TODO: this should be an instant duration lock, how to do
@@ -792,25 +784,24 @@
}
}
} catch (TreeIndexException e) {
- //e.printStackTrace();
- if (!e.getHandled()) {
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ if (!ctx.exceptionHandled) {
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
- e.setHandled(true);
+ ctx.exceptionHandled = true;
+ }
+ throw e;
+ } catch (PageAllocationException e) {
+ if (!ctx.exceptionHandled) {
+ releaseLatch(node, ctx, unsafeIsLeaf);
+ bufferCache.unpin(node);
+ ctx.exceptionHandled = true;
}
throw e;
} catch (Exception e) {
- //e.printStackTrace();
- // This could be caused, e.g. by a failure to pin a new node during a split.
- releaseLatch(node, ctx.op, unsafeIsLeaf);
+ releaseLatch(node, ctx, unsafeIsLeaf);
bufferCache.unpin(node);
- BTreeException propException = new BTreeException(e);
- propException.setHandled(true);
- // propagate a BTreeException,
- // indicating that the parent node
- // must not be unlatched and
- // unpinned
- throw propException;
+ BTreeException wrappedException = new BTreeException(e);
+ throw wrappedException;
}
}
@@ -828,7 +819,7 @@
private final ITreeIndexTupleWriter tupleWriter;
public BulkLoadContext(float fillFactor, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
- ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) throws HyracksDataException {
+ ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) throws HyracksDataException, PageAllocationException {
leafFrame.setMultiComparator(cmp);
interiorFrame.setMultiComparator(cmp);
@@ -859,7 +850,7 @@
nodeFrontiers.add(leafFrontier);
}
- private void addLevel() throws HyracksDataException {
+ private void addLevel() throws HyracksDataException, PageAllocationException {
NodeFrontier frontier = new NodeFrontier(tupleWriter.createTupleReference());
frontier.pageId = freePageManager.getFreePage(metaFrame);
frontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, frontier.pageId), true);
@@ -871,7 +862,7 @@
}
}
- private void propagateBulk(BulkLoadContext ctx, int level) throws HyracksDataException {
+ private void propagateBulk(BulkLoadContext ctx, int level) throws HyracksDataException, PageAllocationException {
if (ctx.splitKey.getBuffer() == null)
return;
@@ -917,7 +908,7 @@
// assumes btree has been created and opened
@Override
- public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException, HyracksDataException {
+ public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException, HyracksDataException, PageAllocationException {
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
if (!isEmptyTree(leafFrame)) {
throw new BTreeException("Trying to Bulk-load a non-empty BTree.");
@@ -925,13 +916,12 @@
BulkLoadContext ctx = new BulkLoadContext(fillFactor, leafFrame,
(IBTreeInteriorFrame)interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory().createFrame(), cmp);
- ctx.nodeFrontiers.get(0).lastTuple.setFieldCount(fieldCount);
ctx.splitKey.getTuple().setFieldCount(cmp.getKeyFieldCount());
return ctx;
}
@Override
- public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
+ public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException, PageAllocationException {
BulkLoadContext ctx = (BulkLoadContext) ictx;
NodeFrontier leafFrontier = ctx.nodeFrontiers.get(0);
IBTreeLeafFrame leafFrame = ctx.leafFrame;
@@ -1006,11 +996,9 @@
}
}
- @Override
- public BTreeOpContext createOpContext(IndexOp op) {
- return new BTreeOpContext(op, (IBTreeLeafFrame) leafFrameFactory.createFrame(),
- (IBTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
- .createFrame(), cmp);
+ private BTreeOpContext createOpContext() {
+ return new BTreeOpContext(leafFrameFactory, interiorFrameFactory, freePageManager.getMetaDataFrameFactory()
+ .createFrame(), cmp);
}
public ITreeIndexFrameFactory getInteriorFrameFactory() {
@@ -1122,4 +1110,50 @@
e.printStackTrace();
}
}
+
+ @Override
+ public ITreeIndexAccessor createAccessor() {
+ return new BTreeAccessor(this);
+ }
+
+ private class BTreeAccessor implements ITreeIndexAccessor {
+ private BTree btree;
+ private BTreeOpContext ctx;
+
+ public BTreeAccessor(BTree btree) {
+ this.btree = btree;
+ this.ctx = btree.createOpContext();
+ }
+
+ @Override
+ public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ ctx.reset(IndexOp.INSERT);
+ btree.insert(tuple, ctx);
+ }
+
+ @Override
+ public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ ctx.reset(IndexOp.UPDATE);
+ btree.update(tuple, ctx);
+ }
+
+ @Override
+ public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ ctx.reset(IndexOp.DELETE);
+ btree.delete(tuple, ctx);
+ }
+
+ @Override
+ public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+ TreeIndexException, PageAllocationException {
+ ctx.reset(IndexOp.SEARCH);
+ btree.search(cursor, searchPred, ctx);
+ }
+
+ @Override
+ public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
+ ctx.reset(IndexOp.DISKORDERSCAN);
+ btree.diskOrderScan(cursor, ctx);
+ }
+ }
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
index 82c747c..07c645c 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeOpContext.java
@@ -19,40 +19,45 @@
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IntArrayList;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.LongArrayList;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-public final class BTreeOpContext implements IIndexOpContext {
- private final int INIT_ARRAYLIST_SIZE = 6;
- public final IBTreeLeafFrame leafFrame;
- public final IBTreeInteriorFrame interiorFrame;
- public final ITreeIndexMetaDataFrame metaFrame;
+public class BTreeOpContext implements IIndexOpContext {
+ private final int INIT_ARRAYLIST_SIZE = 6;
+ protected ITreeIndexFrameFactory leafFrameFactory;
+ protected ITreeIndexFrameFactory interiorFrameFactory;
+ public IBTreeLeafFrame leafFrame;
+ public IBTreeInteriorFrame interiorFrame;
+ public ITreeIndexMetaDataFrame metaFrame;
public IndexOp op;
public ITreeIndexCursor cursor;
public BTreeCursorInitialState cursorInitialState;
public RangePredicate pred;
- public BTreeSplitKey splitKey;
- public int opRestarts = 0;
+ public BTreeSplitKey splitKey;
public LongArrayList pageLsns;
public IntArrayList smPages;
public IntArrayList freePages;
-
- public BTreeOpContext(IndexOp op, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame,
- ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) {
+ public int opRestarts = 0;
+ public boolean exceptionHandled;
+
+ public BTreeOpContext(ITreeIndexFrameFactory leafFrameFactory, ITreeIndexFrameFactory interiorFrameFactory,
+ ITreeIndexMetaDataFrame metaFrame, MultiComparator cmp) {
+ this.leafFrameFactory = leafFrameFactory;
+ this.leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
if (leafFrame != null) {
- leafFrame.setMultiComparator(cmp);
+ leafFrame.setMultiComparator(cmp);
}
- this.leafFrame = leafFrame;
+ this.interiorFrameFactory = interiorFrameFactory;
+ this.interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
if (interiorFrame != null) {
- interiorFrame.setMultiComparator(cmp);
+ interiorFrame.setMultiComparator(cmp);
}
- this.interiorFrame = interiorFrame;
this.metaFrame = metaFrame;
this.pageLsns = new LongArrayList(INIT_ARRAYLIST_SIZE, INIT_ARRAYLIST_SIZE);
- reset(op);
}
public void reset() {
@@ -63,6 +68,7 @@
if (smPages != null)
smPages.clear();
opRestarts = 0;
+ exceptionHandled = false;
}
@Override
@@ -86,6 +92,15 @@
splitKey = new BTreeSplitKey(leafFrame.getTupleWriter().createTupleReference());
}
}
- this.op = newOp;
+ op = newOp;
+ exceptionHandled = false;
+ }
+
+ public IBTreeLeafFrame createLeafFrame() {
+ return (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ }
+
+ public IBTreeInteriorFrame createInteriorFrame() {
+ return (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
}
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 1072bfb..8122314 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -33,7 +33,6 @@
private int fileId = -1;
private ICachedPage page = null;
- private IBTreeLeafFrame frame = null;
private IBufferCache bufferCache = null;
private int tupleIndex = 0;
@@ -46,7 +45,9 @@
private FindTupleNoExactMatchPolicy lowKeyFtp;
private FindTupleNoExactMatchPolicy highKeyFtp;
- private ITreeIndexTupleReference frameTuple;
+ private final IBTreeLeafFrame frame;
+ private final ITreeIndexTupleReference frameTuple;
+ private final boolean exclusiveLatchNodes;
private RangePredicate pred;
private MultiComparator lowKeyCmp;
@@ -54,20 +55,25 @@
private ITupleReference lowKey;
private ITupleReference highKey;
- public BTreeRangeSearchCursor(IBTreeLeafFrame frame) {
+ public BTreeRangeSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) {
this.frame = frame;
- this.frameTuple = frame.createTupleReference();
+ this.frameTuple = frame.createTupleReference();
+ this.exclusiveLatchNodes = exclusiveLatchNodes;
}
@Override
public void close() throws Exception {
if (page != null) {
- page.releaseReadLatch();
+ if (exclusiveLatchNodes) {
+ page.releaseWriteLatch();
+ } else {
+ page.releaseReadLatch();
+ }
bufferCache.unpin(page);
}
tupleIndex = 0;
- page = null;
- pred = null;
+ page = null;
+ pred = null;
}
public ITupleReference getTuple() {
@@ -81,18 +87,20 @@
private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException {
ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false);
- nextLeaf.acquireReadLatch();
-
- page.releaseReadLatch();
+ if (exclusiveLatchNodes) {
+ nextLeaf.acquireWriteLatch();
+ page.releaseWriteLatch();
+ } else {
+ nextLeaf.acquireReadLatch();
+ page.releaseReadLatch();
+ }
bufferCache.unpin(page);
-
page = nextLeaf;
- frame.setPage(page);
+ frame.setPage(page);
}
@Override
public boolean hasNext() throws Exception {
-
if (pred.isForward()) {
if (tupleIndex >= frame.getTupleCount()) {
int nextLeafPage = frame.getNextLeaf();
@@ -142,42 +150,45 @@
}
private int getLowKeyIndex() throws HyracksDataException {
- int index;
- if (lowKey == null)
- index = 0;
- else {
- index = frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lowKeyFtm, lowKeyFtp);
- if (pred.lowKeyInclusive) {
- index++;
- } else {
- if (index < 0)
- index = frame.getTupleCount();
+ if (lowKey == null) {
+ return 0;
+ }
+ int index = frame.findTupleIndex(lowKey, frameTuple, lowKeyCmp, lowKeyFtm, lowKeyFtp);
+ if (pred.lowKeyInclusive) {
+ index++;
+ } else {
+ if (index < 0) {
+ index = frame.getTupleCount();
}
}
return index;
}
private int getHighKeyIndex() throws HyracksDataException {
- int index;
- if (highKey == null)
- index = frame.getTupleCount() - 1;
- else {
- index = frame.findTupleIndex(highKey, frameTuple, highKeyCmp, highKeyFtm, highKeyFtp);
- if (pred.highKeyInclusive) {
- if (index < 0)
- index = frame.getTupleCount() - 1;
- else
- index--;
+ if (highKey == null) {
+ return frame.getTupleCount() - 1;
+ }
+ int index = frame.findTupleIndex(highKey, frameTuple, highKeyCmp, highKeyFtm, highKeyFtp);
+ if (pred.highKeyInclusive) {
+ if (index < 0) {
+ index = frame.getTupleCount() - 1;
+ }
+ else {
+ index--;
}
}
return index;
}
@Override
- public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws Exception {
+ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
// in case open is called multiple times without closing
if (page != null) {
- page.releaseReadLatch();
+ if (exclusiveLatchNodes) {
+ page.releaseWriteLatch();
+ } else {
+ page.releaseReadLatch();
+ }
bufferCache.unpin(page);
}
@@ -220,10 +231,10 @@
@Override
public void reset() {
try {
- close();
- } catch (Exception e) {
- e.printStackTrace();
- }
+ close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
}
@Override
@@ -235,4 +246,9 @@
public void setFileId(int fileId) {
this.fileId = fileId;
}
+
+ @Override
+ public boolean exclusiveLatchNodes() {
+ return exclusiveLatchNodes;
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
index 79945b6..625fa02 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeUtils.java
@@ -1,7 +1,6 @@
package edu.uci.ics.hyracks.storage.am.btree.util;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeException;
@@ -59,12 +58,4 @@
}
}
}
-
- public static MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
- IBinaryComparator[] cmps = new IBinaryComparator[cmpFactories.length];
- for (int i = 0; i < cmpFactories.length; i++) {
- cmps[i] = cmpFactories[i].createBinaryComparator();
- }
- return new MultiComparator(cmps);
- }
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java
index 045ff9d..a7901c8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IFreePageManager.java
@@ -4,7 +4,7 @@
public interface IFreePageManager {
public int getFreePage(ITreeIndexMetaDataFrame metaFrame)
- throws HyracksDataException;
+ throws HyracksDataException, PageAllocationException;
public void addFreePage(ITreeIndexMetaDataFrame metaFrame, int freePage)
throws HyracksDataException;
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
index 83d0639..46d22d1 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndex.java
@@ -17,103 +17,25 @@
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.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
/**
* Interface describing the operations of tree-based index structures. Indexes
* implementing this interface can easily reuse the tree index operators for
* dataflow. We assume that indexes store tuples with a fixed number of fields.
+ * Users must perform operations on an ITreeIndex via an ITreeIndexAccessor.
*/
-public interface ITreeIndex {
+public interface ITreeIndex extends IIndex {
/**
- * Initializes the persistent state of a tree index, e.g., the root page,
- * and metadata pages.
+ * Creates an index accessor for performing operations on this index.
+ * (insert/delete/update/search/diskorderscan). An ITreeIndexAccessor is not
+ * thread safe, but different ITreeIndexAccessors can concurrently operate
+ * on the same ITreeIndex
*
- * @param indexFileId
- * The file id to use for this index.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
+ * @returns ITreeIndexAccessor A tree index accessor for this tree.
*/
- public void create(int indexFileId) throws HyracksDataException;
-
- /**
- * Opens the tree index backed by the given file id.
- *
- * @param indexFileId
- * The file id backing this index.
- */
- public void open(int indexFileId);
-
- /**
- * Closes the tree index.
- */
- public void close();
-
- /**
- * Creates an operation context for a given index operation
- * (insert/delete/update/search/diskorderscan). An operation context
- * maintains a cache of objects used during the traversal of the tree index.
- * The context is intended to be reused for multiple subsequent operations
- * by the same user/thread. An index operation context is stateful, and
- * therefore, should not be shared among two threads.
- *
- * @param indexOp
- * Intended index operation.
- *
- * @returns IITreeIndexOpContext Operation context for the desired index
- * operation.
- */
- public IIndexOpContext createOpContext(IndexOp op);
-
- /**
- * Inserts the given tuple into the index using an existing operation
- * context.
- *
- * @param tuple
- * Tuple to be inserted.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- * @throws TreeIndexException
- * If an index-specific constraint is violated, e.g., the key
- * already exists.
- */
- public void insert(ITupleReference tuple, IIndexOpContext ictx)
- throws HyracksDataException, TreeIndexException;
-
- /**
- * Updates the tuple in the index matching the given tuple with the new
- * contents in the given tuple.
- *
- * @param tuple
- * Tuple whose match in the index is to be update with the given
- * tuples contents.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- * @throws TreeIndexException
- * If there is no matching tuple in the index.
- */
- public void update(ITupleReference tuple, IIndexOpContext ictx)
- throws HyracksDataException, TreeIndexException;
-
- /**
- * Deletes the tuple in the index matching the given tuple.
- *
- * @param tuple
- * Tuple to be deleted.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- * @throws TreeIndexException
- * If there is no matching tuple in the index.
- */
- public void delete(ITupleReference tuple, IIndexOpContext ictx)
- throws HyracksDataException, TreeIndexException;
+ public ITreeIndexAccessor createAccessor();
/**
* Prepares the index for bulk loading, returning a bulk load context. The
@@ -125,10 +47,12 @@
* If the BufferCache throws while un/pinning or un/latching.
* @throws TreeIndexException
* If the tree is not empty.
+ * @throws PageAllocationException
* @returns A new context for bulk loading, required for appending tuples.
*/
- public IIndexBulkLoadContext beginBulkLoad(float fillFactor) throws TreeIndexException,
- HyracksDataException;
+ public IIndexBulkLoadContext beginBulkLoad(float fillFactor)
+ throws TreeIndexException, HyracksDataException,
+ PageAllocationException;
/**
* Append a tuple to the index in the context of a bulk load.
@@ -139,9 +63,11 @@
* Existing bulk load context.
* @throws HyracksDataException
* If the BufferCache throws while un/pinning or un/latching.
+ * @throws PageAllocationException
*/
public void bulkLoadAddTuple(ITupleReference tuple,
- IIndexBulkLoadContext ictx) throws HyracksDataException;
+ IIndexBulkLoadContext ictx) throws HyracksDataException,
+ PageAllocationException;
/**
* Finalize the bulk loading operation in the given context.
@@ -150,24 +76,12 @@
* Existing bulk load context to be finalized.
* @throws HyracksDataException
* If the BufferCache throws while un/pinning or un/latching.
+ * @throws PageAllocationException
*/
public void endBulkLoad(IIndexBulkLoadContext ictx)
- throws HyracksDataException;
+ throws HyracksDataException, PageAllocationException;
/**
- * Open the given cursor for a disk-order scan, positioning the cursor to
- * the first leaf tuple.
- *
- * @param icursor
- * Cursor to be opened for disk-order scanning.
- * @param ictx
- * Existing operation context.
- * @throws HyracksDataException
- * If the BufferCache throws while un/pinning or un/latching.
- */
- public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException;
-
- /**
* @return The index's leaf frame factory.
*/
public ITreeIndexFrameFactory getLeafFrameFactory();
@@ -176,7 +90,7 @@
* @return The index's interior frame factory.
*/
public ITreeIndexFrameFactory getInteriorFrameFactory();
-
+
/**
* @return The index's free page manager.
*/
@@ -186,7 +100,7 @@
* @return The number of fields tuples of this index have.
*/
public int getFieldCount();
-
+
/**
* @return The current root page id of this index.
*/
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java
new file mode 100644
index 0000000..1e679b2
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexAccessor.java
@@ -0,0 +1,101 @@
+/*
+ * 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.api;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Client handle for performing operations
+ * (insert/delete/update/search/diskorderscan) on an ITreeIndex. An
+ * ITreeIndexAccessor is not thread safe, but different ITreeIndexAccessors can
+ * concurrently operate on the same ITreeIndex (i.e., the ITreeIndex must allow
+ * concurrent operations).
+ */
+public interface ITreeIndexAccessor {
+ /**
+ * Inserts the given tuple.
+ *
+ * @param tuple
+ * Tuple to be inserted.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * If an index-specific constraint is violated, e.g., the key
+ * already exists.
+ * @throws PageAllocationException
+ */
+ public void insert(ITupleReference tuple) throws HyracksDataException,
+ TreeIndexException, PageAllocationException;
+
+ /**
+ * Updates the tuple in the index matching the given tuple with the new
+ * contents in the given tuple.
+ *
+ * @param tuple
+ * Tuple whose match in the index is to be update with the given
+ * tuples contents.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * If there is no matching tuple in the index.
+ * @throws PageAllocationException
+ */
+ public void update(ITupleReference tuple) throws HyracksDataException,
+ TreeIndexException, PageAllocationException;
+
+ /**
+ * Deletes the tuple in the index matching the given tuple.
+ *
+ * @param tuple
+ * Tuple to be deleted.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * If there is no matching tuple in the index.
+ * @throws PageAllocationException
+ */
+ public void delete(ITupleReference tuple) throws HyracksDataException,
+ TreeIndexException, PageAllocationException;
+
+ /**
+ * Open the given cursor for an index search using the given predicate as
+ * search condition.
+ *
+ * @param icursor
+ * Cursor over the index entries satisfying searchPred.
+ * @param searchPred
+ * Search condition.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ * @throws TreeIndexException
+ * @throws PageAllocationException
+ */
+ public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred)
+ throws HyracksDataException, TreeIndexException, PageAllocationException;
+
+ /**
+ * Open the given cursor for a disk-order scan, positioning the cursor to
+ * the first leaf tuple.
+ *
+ * @param icursor
+ * Cursor to be opened for disk-order scanning.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ */
+ public void diskOrderScan(ITreeIndexCursor cursor)
+ throws HyracksDataException;
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
index 56dfd47..d3ce3867 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.api;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
@@ -27,7 +28,7 @@
public void next() throws Exception;
public void open(ICursorInitialState initialState,
- ISearchPredicate searchPred) throws Exception;
+ ISearchPredicate searchPred) throws HyracksDataException;
public ICachedPage getPage();
@@ -38,4 +39,7 @@
public void setFileId(int fileId);
public ITupleReference getTuple();
+
+ // For allowing updates.
+ public boolean exclusiveLatchNodes();
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdater.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdater.java
new file mode 100644
index 0000000..e201cc3
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdater.java
@@ -0,0 +1,29 @@
+/*
+ * 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.api;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Interface for updating a tuple. Warning: By convention, this interface
+ * assumes that the modifications do not change the size of the tuple, and that
+ * it does not change keys (e.g., BTree keys). This interface is used to
+ * implement update scans.
+ *
+ */
+public interface ITupleUpdater {
+ public void updateTuple(ITupleReference tuple);
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdaterFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdaterFactory.java
new file mode 100644
index 0000000..ee20b6c
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITupleUpdaterFactory.java
@@ -0,0 +1,22 @@
+/*
+ * 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.api;
+
+import java.io.Serializable;
+
+public interface ITupleUpdaterFactory extends Serializable {
+ public ITupleUpdater createTupleUpdater();
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/PageAllocationException.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/PageAllocationException.java
new file mode 100644
index 0000000..e6eec66
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/PageAllocationException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.api;
+
+public class PageAllocationException extends Exception {
+
+ private static final long serialVersionUID = 1L;
+
+ public PageAllocationException(Throwable cause) {
+ super(cause);
+ }
+
+ public PageAllocationException(String message) {
+ super(message);
+ }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index 3a0710e..459ab57 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -21,13 +21,12 @@
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public abstract class AbstractTreeIndexOperatorDescriptor extends
AbstractSingleActivityOperatorDescriptor implements
- ITreeIndexOperatorDescriptorHelper {
+ ITreeIndexOperatorDescriptor {
private static final long serialVersionUID = 1L;
@@ -39,36 +38,36 @@
protected final ITreeIndexFrameFactory leafFrameFactory;
protected final IStorageManagerInterface storageManager;
- protected final IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
+ protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
protected final ITypeTrait[] typeTraits;
-
- protected final ITreeIndexOpHelperFactory opHelperFactory;
+ protected final IIndexDataflowHelperFactory dataflowHelperFactory;
public AbstractTreeIndexOperatorDescriptor(JobSpecification spec,
int inputArity, int outputArity, RecordDescriptor recDesc,
IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories,
- ITreeIndexOpHelperFactory opHelperFactory) {
+ IIndexDataflowHelperFactory dataflowHelperFactory) {
super(spec, inputArity, outputArity);
this.fileSplitProvider = fileSplitProvider;
this.storageManager = storageManager;
- this.treeIndexRegistryProvider = treeIndexRegistryProvider;
+ this.indexRegistryProvider = indexRegistryProvider;
this.interiorFrameFactory = interiorFrameFactory;
this.leafFrameFactory = leafFrameFactory;
this.typeTraits = typeTraits;
this.comparatorFactories = comparatorFactories;
- this.opHelperFactory = opHelperFactory;
- if (outputArity > 0)
+ this.dataflowHelperFactory = dataflowHelperFactory;
+ if (outputArity > 0) {
recordDescriptors[0] = recDesc;
+ }
}
@Override
- public IFileSplitProvider getTreeIndexFileSplitProvider() {
+ public IFileSplitProvider getFileSplitProvider() {
return fileSplitProvider;
}
@@ -83,11 +82,6 @@
}
@Override
- public int getTreeIndexFieldCount() {
- return typeTraits.length;
- }
-
- @Override
public ITreeIndexFrameFactory getTreeIndexInteriorFactory() {
return interiorFrameFactory;
}
@@ -103,8 +97,8 @@
}
@Override
- public IIndexRegistryProvider<ITreeIndex> getTreeIndexRegistryProvider() {
- return treeIndexRegistryProvider;
+ public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
+ return indexRegistryProvider;
}
@Override
@@ -113,7 +107,7 @@
}
@Override
- public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory() {
- return opHelperFactory;
+ public IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
+ return dataflowHelperFactory;
}
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndex.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndex.java
new file mode 100644
index 0000000..38b275b
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndex.java
@@ -0,0 +1,47 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Interface describing the operations common to all indexes.
+ */
+public interface IIndex {
+ /**
+ * Initializes the persistent state of an index, e.g., the root page,
+ * and metadata pages.
+ *
+ * @param indexFileId
+ * The file id to use for this index.
+ * @throws HyracksDataException
+ * If the BufferCache throws while un/pinning or un/latching.
+ */
+ public void create(int indexFileId) throws HyracksDataException;
+
+ /**
+ * Opens the index backed by the given file id.
+ *
+ * @param indexFileId
+ * The file id backing this index.
+ */
+ public void open(int indexFileId);
+
+ /**
+ * Closes the index.
+ */
+ public void close();
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.java
new file mode 100644
index 0000000..a42cf5f
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexDataflowHelperFactory.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.dataflow;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface IIndexDataflowHelperFactory extends Serializable {
+ public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc,
+ final IHyracksTaskContext ctx, int partition, boolean createIfNotExists);
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
new file mode 100644
index 0000000..edf2bba
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public interface IIndexOperatorDescriptor extends IActivity {
+ public IFileSplitProvider getFileSplitProvider();
+
+ public IStorageManagerInterface getStorageManager();
+
+ public IIndexRegistryProvider<IIndex> getIndexRegistryProvider();
+
+ public RecordDescriptor getRecordDescriptor();
+
+ public IIndexDataflowHelperFactory getIndexDataflowHelperFactory();
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
deleted file mode 100644
index 1f734f4..0000000
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
+++ /dev/null
@@ -1,12 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.dataflow;
-
-import java.io.Serializable;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-
-public interface ITreeIndexOpHelperFactory extends Serializable {
- public TreeIndexOpHelper createTreeIndexOpHelper(
- ITreeIndexOperatorDescriptorHelper opDesc,
- final IHyracksTaskContext ctx, int partition,
- IndexHelperOpenMode mode);
-}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
new file mode 100644
index 0000000..9bbca56
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
@@ -0,0 +1,31 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+
+public interface ITreeIndexOperatorDescriptor extends IIndexOperatorDescriptor {
+ public IBinaryComparatorFactory[] getTreeIndexComparatorFactories();
+
+ public ITreeIndexFrameFactory getTreeIndexInteriorFactory();
+
+ public ITreeIndexFrameFactory getTreeIndexLeafFactory();
+
+ public ITypeTrait[] getTreeIndexTypeTraits();
+
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
deleted file mode 100644
index 468c04e..0000000
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
+++ /dev/null
@@ -1,33 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.dataflow;
-
-import edu.uci.ics.hyracks.api.dataflow.IActivity;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public interface ITreeIndexOperatorDescriptorHelper extends IActivity {
- public IFileSplitProvider getTreeIndexFileSplitProvider();
-
- public IBinaryComparatorFactory[] getTreeIndexComparatorFactories();
-
- // TODO: Is this really needed?
- public ITypeTrait[] getTreeIndexTypeTraits();
-
- public int getTreeIndexFieldCount();
-
- public ITreeIndexFrameFactory getTreeIndexInteriorFactory();
-
- public ITreeIndexFrameFactory getTreeIndexLeafFactory();
-
- public IStorageManagerInterface getStorageManager();
-
- public IIndexRegistryProvider<ITreeIndex> getTreeIndexRegistryProvider();
-
- public RecordDescriptor getRecordDescriptor();
-
- public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory();
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
new file mode 100644
index 0000000..f0adb0e
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexDataflowHelper.java
@@ -0,0 +1,114 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+
+public abstract class IndexDataflowHelper {
+ protected IIndex index;
+ protected int indexFileId = -1;
+ protected int partition;
+
+ protected final IIndexOperatorDescriptor opDesc;
+ protected final IHyracksTaskContext ctx;
+ protected final boolean createIfNotExists;
+
+ public IndexDataflowHelper(IIndexOperatorDescriptor opDesc, final IHyracksTaskContext ctx,
+ int partition, boolean createIfNotExists) {
+ this.opDesc = opDesc;
+ this.ctx = ctx;
+ this.partition = partition;
+ this.createIfNotExists = createIfNotExists;
+ }
+
+ public void init() throws HyracksDataException {
+ IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+ IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
+
+ FileReference f = getFilereference();
+ int fileId = -1;
+ synchronized (fileMapProvider) {
+ boolean fileIsMapped = fileMapProvider.isMapped(f);
+ if (!fileIsMapped) {
+ bufferCache.createFile(f);
+ }
+ fileId = fileMapProvider.lookupFileId(f);
+ try {
+ bufferCache.openFile(fileId);
+ } catch (HyracksDataException e) {
+ // Revert state of buffer cache since file failed to open.
+ if (!fileIsMapped) {
+ bufferCache.deleteFile(fileId);
+ }
+ throw e;
+ }
+ }
+
+ // Only set indexFileId member when openFile() succeeds,
+ // otherwise deinit() will try to close the file that failed to open
+ indexFileId = fileId;
+ IndexRegistry<IIndex> indexRegistry = opDesc.getIndexRegistryProvider().getRegistry(ctx);
+ // Create new index instance and register it.
+ synchronized (indexRegistry) {
+ // Check if the index has already been registered.
+ index = indexRegistry.get(indexFileId);
+ if (index != null) {
+ return;
+ }
+ index = createIndexInstance();
+ if (createIfNotExists) {
+ index.create(indexFileId);
+ }
+ index.open(indexFileId);
+ indexRegistry.register(indexFileId, index);
+ }
+ }
+
+ public abstract IIndex createIndexInstance() throws HyracksDataException;
+
+ public FileReference getFilereference() {
+ IFileSplitProvider fileSplitProvider = opDesc.getFileSplitProvider();
+ return fileSplitProvider.getFileSplits()[partition].getLocalFile();
+ }
+
+ public void deinit() throws HyracksDataException {
+ if (indexFileId != -1) {
+ IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
+ bufferCache.closeFile(indexFileId);
+ }
+ }
+
+ public IIndex getIndex() {
+ return index;
+ }
+
+ public IHyracksTaskContext getHyracksTaskContext() {
+ return ctx;
+ }
+
+ public IIndexOperatorDescriptor getOperatorDescriptor() {
+ return opDesc;
+ }
+
+ public int getIndexFileId() {
+ return indexFileId;
+ }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexHelperOpenMode.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexHelperOpenMode.java
deleted file mode 100644
index 0b27a0e..0000000
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexHelperOpenMode.java
+++ /dev/null
@@ -1,5 +0,0 @@
-package edu.uci.ics.hyracks.storage.am.common.dataflow;
-
-public enum IndexHelperOpenMode {
- OPEN, CREATE, ENLIST
-}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java
index df372f4c..5f3b0b3 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexRegistry.java
@@ -16,26 +16,15 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import java.util.HashMap;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
public class IndexRegistry<IndexType> {
private HashMap<Integer, IndexType> map = new HashMap<Integer, IndexType>();
- private Lock registryLock = new ReentrantLock();
public IndexType get(int fileId) {
return map.get(fileId);
}
- public void lock() {
- registryLock.lock();
- }
-
- public void unlock() {
- registryLock.unlock();
- }
-
public void register(int fileId, IndexType index) {
map.put(fileId, index);
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index a58522f..c5b5355 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -22,40 +22,31 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-public class TreeIndexBulkLoadOperatorDescriptor extends
- AbstractTreeIndexOperatorDescriptor {
+public class TreeIndexBulkLoadOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- private final int[] fieldPermutation;
- private final float fillFactor;
+ private final int[] fieldPermutation;
+ private final float fillFactor;
- public TreeIndexBulkLoadOperatorDescriptor(JobSpecification spec,
- IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- IFileSplitProvider fileSplitProvider,
- ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories,
- int[] fieldPermutation, float fillFactor,
- ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 1, 0, null, storageManager, treeIndexRegistryProvider,
- fileSplitProvider, interiorFrameFactory, leafFrameFactory,
- typeTraits, comparatorFactories,
- opHelperFactory);
- this.fieldPermutation = fieldPermutation;
- this.fillFactor = fillFactor;
- }
+ public TreeIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
+ ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+ ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation,
+ float fillFactor, IIndexDataflowHelperFactory dataflowHelperFactory) {
+ super(spec, 1, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+ leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
+ this.fieldPermutation = fieldPermutation;
+ this.fillFactor = fillFactor;
+ }
- @Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions) {
- return new TreeIndexBulkLoadOperatorNodePushable(this, ctx, partition,
- fieldPermutation, fillFactor, recordDescProvider);
- }
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new TreeIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
+ recordDescProvider);
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
index 63edc06..bc2d67e 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
@@ -23,71 +23,73 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
-public class TreeIndexBulkLoadOperatorNodePushable extends
- AbstractUnaryInputSinkOperatorNodePushable {
- private float fillFactor;
- private final TreeIndexOpHelper treeIndexOpHelper;
- private FrameTupleAccessor accessor;
- private IIndexBulkLoadContext bulkLoadCtx;
+public class TreeIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+ private float fillFactor;
+ private final TreeIndexDataflowHelper treeIndexHelper;
+ private FrameTupleAccessor accessor;
+ private IIndexBulkLoadContext bulkLoadCtx;
+ private ITreeIndex treeIndex;
- private IRecordDescriptorProvider recordDescProvider;
+ private IRecordDescriptorProvider recordDescProvider;
- private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+ private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
- public TreeIndexBulkLoadOperatorNodePushable(
- AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
- float fillFactor, IRecordDescriptorProvider recordDescProvider) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory()
- .createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.CREATE);
- this.fillFactor = fillFactor;
- this.recordDescProvider = recordDescProvider;
- tuple.setFieldPermutation(fieldPermutation);
- }
+ public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
+ treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ opDesc, ctx, partition, true);
+ this.fillFactor = fillFactor;
+ this.recordDescProvider = recordDescProvider;
+ tuple.setFieldPermutation(fieldPermutation);
+ }
- @Override
- public void open() throws HyracksDataException {
- AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
- .getOperatorDescriptor();
- RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(
- opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper
- .getHyracksTaskContext().getFrameSize(), recDesc);
- try {
- treeIndexOpHelper.init();
- treeIndexOpHelper.getTreeIndex().open(
- treeIndexOpHelper.getIndexFileId());
- bulkLoadCtx = treeIndexOpHelper.getTreeIndex().beginBulkLoad(fillFactor);
- } catch (Exception e) {
- // cleanup in case of failure
- treeIndexOpHelper.deinit();
- throw new HyracksDataException(e);
- }
- }
+ @Override
+ public void open() throws HyracksDataException {
+ AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
+ .getOperatorDescriptor();
+ RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ try {
+ treeIndexHelper.init();
+ treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
+ treeIndex.open(treeIndexHelper.getIndexFileId());
+ bulkLoadCtx = treeIndex.beginBulkLoad(fillFactor);
+ } catch (Exception e) {
+ // cleanup in case of failure
+ treeIndexHelper.deinit();
+ throw new HyracksDataException(e);
+ }
+ }
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- accessor.reset(buffer);
- int tupleCount = accessor.getTupleCount();
- for (int i = 0; i < tupleCount; i++) {
- tuple.reset(accessor, i);
- treeIndexOpHelper.getTreeIndex().bulkLoadAddTuple(tuple,
- bulkLoadCtx);
- }
- }
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ try {
+ treeIndex.bulkLoadAddTuple(tuple, bulkLoadCtx);
+ } catch (PageAllocationException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
- @Override
- public void close() throws HyracksDataException {
- try {
- treeIndexOpHelper.getTreeIndex().endBulkLoad(bulkLoadCtx);
- } finally {
- treeIndexOpHelper.deinit();
- }
- }
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ treeIndex.endBulkLoad(bulkLoadCtx);
+ } catch (PageAllocationException e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexHelper.deinit();
+ }
+ }
- @Override
- public void fail() throws HyracksDataException {
- }
+ @Override
+ public void fail() throws HyracksDataException {
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
new file mode 100644
index 0000000..17c1827
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
@@ -0,0 +1,38 @@
+/*
+ * 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.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
+
+public abstract class TreeIndexDataflowHelper extends IndexDataflowHelper {
+ protected ITreeIndexOperatorDescriptor treeOpDesc;
+ public TreeIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+ boolean createIfNotExists) {
+ super(opDesc, ctx, partition, createIfNotExists);
+ this.treeOpDesc = (ITreeIndexOperatorDescriptor) opDesc;
+ }
+
+ public abstract ITreeIndex createIndexInstance() throws HyracksDataException;
+
+ public ITreeIndexCursor createDiskOrderScanCursor(ITreeIndexFrame leafFrame) throws HyracksDataException {
+ return new TreeDiskOrderScanCursor(leafFrame);
+ }
+}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 9a1fc87..8cc83f7 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -22,7 +22,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -33,14 +32,14 @@
public TreeIndexDiskOrderScanOperatorDescriptor(JobSpecification spec,
RecordDescriptor recDesc, IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
- ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 0, 1, recDesc, storageManager, treeIndexRegistryProvider,
+ IIndexDataflowHelperFactory dataflowHelperFactory) {
+ super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider,
fileSplitProvider, interiorFrameFactory, leafFrameFactory,
- typeTraits, null, opHelperFactory);
+ typeTraits, null, dataflowHelperFactory);
}
@Override
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index ab7b335..d9b4d87 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -24,92 +24,78 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+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.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-public class TreeIndexDiskOrderScanOperatorNodePushable extends
- AbstractUnaryOutputSourceOperatorNodePushable {
- private final TreeIndexOpHelper treeIndexOpHelper;
+public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+ private final TreeIndexDataflowHelper treeIndexHelper;
+ private final ITreeIndexOperatorDescriptor opDesc;
+ private ITreeIndex treeIndex;
- public TreeIndexDiskOrderScanOperatorNodePushable(
- AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksTaskContext ctx, int partition) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory()
- .createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.OPEN);
- }
+ public TreeIndexDiskOrderScanOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition) {
+ treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ opDesc, ctx, partition, false);
+ this.opDesc = opDesc;
+ }
- @Override
- public void initialize() throws HyracksDataException {
+ @Override
+ public void initialize() throws HyracksDataException {
+ ITreeIndexFrame cursorFrame = opDesc.getTreeIndexLeafFactory().createFrame();
+ ITreeIndexCursor cursor = treeIndexHelper.createDiskOrderScanCursor(cursorFrame);
+ try {
+ treeIndexHelper.init();
+ treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
+ ITreeIndexAccessor indexAccessor = treeIndex.createAccessor();
+ writer.open();
+ try {
+ indexAccessor.diskOrderScan(cursor);
+ int fieldCount = treeIndex.getFieldCount();
+ ByteBuffer frame = treeIndexHelper.getHyracksTaskContext().allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext()
+ .getFrameSize());
+ appender.reset(frame, true);
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ DataOutput dos = tb.getDataOutput();
- ITreeIndexFrame cursorFrame = treeIndexOpHelper.getOperatorDescriptor()
- .getTreeIndexLeafFactory().createFrame();
- ITreeIndexCursor cursor = treeIndexOpHelper
- .createDiskOrderScanCursor(cursorFrame);
- IIndexOpContext diskOrderScanOpCtx = treeIndexOpHelper.getTreeIndex()
- .createOpContext(IndexOp.DISKORDERSCAN);
- try {
+ while (cursor.hasNext()) {
+ tb.reset();
+ cursor.next();
- treeIndexOpHelper.init();
- writer.open();
- try {
- treeIndexOpHelper.getTreeIndex().diskOrderScan(cursor, diskOrderScanOpCtx);
+ ITupleReference frameTuple = cursor.getTuple();
+ for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+ dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
- int fieldCount = treeIndexOpHelper.getTreeIndex()
- .getFieldCount();
- ByteBuffer frame = treeIndexOpHelper.getHyracksTaskContext()
- .allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(
- treeIndexOpHelper.getHyracksTaskContext()
- .getFrameSize());
- appender.reset(frame, true);
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } catch (Exception e) {
+ writer.fail();
+ throw new HyracksDataException(e);
+ } finally {
+ cursor.close();
+ writer.close();
+ }
+ } catch (Exception e) {
+ deinitialize();
+ throw new HyracksDataException(e);
+ }
+ }
- while (cursor.hasNext()) {
- tb.reset();
- cursor.next();
-
- ITupleReference frameTuple = cursor.getTuple();
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i),
- frameTuple.getFieldStart(i),
- frameTuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
-
- if (!appender.append(tb.getFieldEndOffsets(),
- tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(frame, writer);
- appender.reset(frame, true);
- if (!appender.append(tb.getFieldEndOffsets(),
- tb.getByteArray(), 0, tb.getSize())) {
- throw new IllegalStateException();
- }
- }
- }
-
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
- } catch (Exception e) {
- writer.fail();
- throw new HyracksDataException(e);
- } finally {
- cursor.close();
- writer.close();
- }
-
- } catch (Exception e) {
- deinitialize();
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- public void deinitialize() throws HyracksDataException {
- treeIndexOpHelper.deinit();
- }
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ treeIndexHelper.deinit();
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
index 6614fba..65b799e 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
@@ -30,87 +30,75 @@
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-public class TreeIndexDropOperatorNodePushable extends
- AbstractOperatorNodePushable {
- private static final Logger LOGGER = Logger
- .getLogger(TreeIndexDropOperatorNodePushable.class.getName());
+public class TreeIndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
+ private static final Logger LOGGER = Logger.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
- private final IHyracksTaskContext ctx;
- private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
- private IStorageManagerInterface storageManager;
- private IFileSplitProvider fileSplitProvider;
- private int partition;
+ private final IHyracksTaskContext ctx;
+ private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
+ private IStorageManagerInterface storageManager;
+ private IFileSplitProvider fileSplitProvider;
+ private int partition;
- public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx,
- IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- IFileSplitProvider fileSplitProvider, int partition) {
- this.ctx = ctx;
- this.storageManager = storageManager;
- this.treeIndexRegistryProvider = treeIndexRegistryProvider;
- this.fileSplitProvider = fileSplitProvider;
- this.partition = partition;
- }
+ public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx, IStorageManagerInterface storageManager,
+ IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
+ int partition) {
+ this.ctx = ctx;
+ this.storageManager = storageManager;
+ this.treeIndexRegistryProvider = treeIndexRegistryProvider;
+ this.fileSplitProvider = fileSplitProvider;
+ this.partition = partition;
+ }
- @Override
- public void deinitialize() throws HyracksDataException {
- }
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ }
- @Override
- public int getInputArity() {
- return 0;
- }
+ @Override
+ public int getInputArity() {
+ return 0;
+ }
- @Override
- public IFrameWriter getInputFrameWriter(int index) {
- return null;
- }
+ @Override
+ public IFrameWriter getInputFrameWriter(int index) {
+ return null;
+ }
- @Override
- public void initialize() throws HyracksDataException {
- try {
+ @Override
+ public void initialize() throws HyracksDataException {
+ try {
+ IndexRegistry<ITreeIndex> treeIndexRegistry = treeIndexRegistryProvider.getRegistry(ctx);
+ IBufferCache bufferCache = storageManager.getBufferCache(ctx);
+ IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
+
+ FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
+ int indexFileId = -1;
+ synchronized (fileMapProvider) {
+ boolean fileIsMapped = fileMapProvider.isMapped(f);
+ if (!fileIsMapped) {
+ throw new HyracksDataException("Cannot drop Tree with name " + f.toString()
+ + ". No file mapping exists.");
+ }
+ indexFileId = fileMapProvider.lookupFileId(f);
+ }
+ // Unregister tree instance.
+ synchronized (treeIndexRegistry) {
+ treeIndexRegistry.unregister(indexFileId);
+ }
- IndexRegistry<ITreeIndex> treeIndexRegistry = treeIndexRegistryProvider
- .getRegistry(ctx);
- IBufferCache bufferCache = storageManager.getBufferCache(ctx);
- IFileMapProvider fileMapProvider = storageManager
- .getFileMapProvider(ctx);
+ // remove name to id mapping
+ bufferCache.deleteFile(indexFileId);
+ }
+ // TODO: for the time being we don't throw,
+ // with proper exception handling (no hanging job problem) we should
+ // throw
+ catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Tree Drop Operator Failed Due To Exception: " + e.getMessage());
+ }
+ }
+ }
- FileReference f = fileSplitProvider.getFileSplits()[partition]
- .getLocalFile();
-
- boolean fileIsMapped = fileMapProvider.isMapped(f);
- if (!fileIsMapped) {
- throw new HyracksDataException("Cannot drop Tree with name "
- + f.toString() + ". No file mapping exists.");
- }
-
- int indexFileId = fileMapProvider.lookupFileId(f);
-
- // unregister tree instance
- treeIndexRegistry.lock();
- try {
- treeIndexRegistry.unregister(indexFileId);
- } finally {
- treeIndexRegistry.unlock();
- }
-
- // remove name to id mapping
- bufferCache.deleteFile(indexFileId);
- }
- // TODO: for the time being we don't throw,
- // with proper exception handling (no hanging job problem) we should
- // throw
- catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Tree Drop Operator Failed Due To Exception: "
- + e.getMessage());
- }
- }
- }
-
- @Override
- public void setOutputFrameWriter(int index, IFrameWriter writer,
- RecordDescriptor recordDesc) {
- }
+ @Override
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
deleted file mode 100644
index 15e230c..0000000
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-// re-create in-memory state for a tree index that has already been built (i.e., the file exists):
-// 1. register files in file manager (FileManager)
-// 2. create file mappings (FileMappingProvider)
-// 3. register tree index instance (IndexRegistry)
-
-public class TreeIndexFileEnlistmentOperatorDescriptor extends
- AbstractTreeIndexOperatorDescriptor {
-
- private static final long serialVersionUID = 1L;
-
- public TreeIndexFileEnlistmentOperatorDescriptor(JobSpecification spec,
- RecordDescriptor recDesc, IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- IFileSplitProvider fileSplitProvider,
- ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories,
- ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 0, 0, recDesc, storageManager, treeIndexRegistryProvider,
- fileSplitProvider, interiorFrameFactory, leafFrameFactory,
- typeTraits, comparatorFactories, opHelperFactory);
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider,
- int partition, int partitions) throws HyracksDataException {
- return new TreeIndexFileEnlistmentOperatorNodePushable(this, ctx,
- partition);
- }
-
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
deleted file mode 100644
index 124e2cc..0000000
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.dataflow;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
-
-public class TreeIndexFileEnlistmentOperatorNodePushable extends
- AbstractOperatorNodePushable {
-
- private final TreeIndexOpHelper treeIndexOpHelper;
-
- public TreeIndexFileEnlistmentOperatorNodePushable(
- AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksTaskContext ctx, int partition) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory()
- .createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.ENLIST);
- }
-
- @Override
- public void deinitialize() throws HyracksDataException {
- }
-
- @Override
- public int getInputArity() {
- return 0;
- }
-
- @Override
- public IFrameWriter getInputFrameWriter(int index) {
- return null;
- }
-
- @Override
- public void initialize() throws HyracksDataException {
- try {
- treeIndexOpHelper.init();
- } finally {
- treeIndexOpHelper.deinit();
- }
- }
-
- @Override
- public void setOutputFrameWriter(int index, IFrameWriter writer,
- RecordDescriptor recordDesc) {
- }
-}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 2c525dd..50fb3ea 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -23,7 +23,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -39,17 +38,17 @@
public TreeIndexInsertUpdateDeleteOperatorDescriptor(JobSpecification spec,
RecordDescriptor recDesc, IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider,
ITreeIndexFrameFactory interiorFrameFactory,
ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories,
int[] fieldPermutation, IndexOp op,
- ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider,
+ IIndexDataflowHelperFactory dataflowHelperFactory) {
+ super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider,
fileSplitProvider, interiorFrameFactory, leafFrameFactory,
typeTraits, comparatorFactories,
- opHelperFactory);
+ dataflowHelperFactory);
this.fieldPermutation = fieldPermutation;
this.op = op;
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index 0ae513b..bf78249 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -23,110 +23,97 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
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.ophelpers.IndexOp;
-public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends
- AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private final TreeIndexOpHelper treeIndexOpHelper;
- private FrameTupleAccessor accessor;
- private final IRecordDescriptorProvider recordDescProvider;
- private final IndexOp op;
- private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
- private ByteBuffer writeBuffer;
- private IIndexOpContext opCtx;
+public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+ private final TreeIndexDataflowHelper treeIndexHelper;
+ private FrameTupleAccessor accessor;
+ private final IRecordDescriptorProvider recordDescProvider;
+ private final IndexOp op;
+ private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+ private ByteBuffer writeBuffer;
+ private ITreeIndexAccessor indexAccessor;
- public TreeIndexInsertUpdateDeleteOperatorNodePushable(
- AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
- IRecordDescriptorProvider recordDescProvider, IndexOp op) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory()
- .createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.OPEN);
- this.recordDescProvider = recordDescProvider;
- this.op = op;
- tuple.setFieldPermutation(fieldPermutation);
- }
+ public TreeIndexInsertUpdateDeleteOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
+ IRecordDescriptorProvider recordDescProvider, IndexOp op) {
+ // Only create the if insert operation is an insert.
+ boolean createIfNotExists = (op == IndexOp.INSERT);
+ treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ opDesc, ctx, partition, createIfNotExists);
+ this.recordDescProvider = recordDescProvider;
+ this.op = op;
+ tuple.setFieldPermutation(fieldPermutation);
+ }
- @Override
- public void open() throws HyracksDataException {
- AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
- .getOperatorDescriptor();
- RecordDescriptor inputRecDesc = recordDescProvider
- .getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper
- .getHyracksTaskContext().getFrameSize(), inputRecDesc);
- writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
- writer.open();
- try {
- treeIndexOpHelper.init();
- treeIndexOpHelper.getTreeIndex().open(
- treeIndexOpHelper.getIndexFileId());
- opCtx = treeIndexOpHelper.getTreeIndex().createOpContext(op);
- } catch (Exception e) {
- // cleanup in case of failure
- treeIndexOpHelper.deinit();
- throw new HyracksDataException(e);
- }
- }
+ @Override
+ public void open() throws HyracksDataException {
+ AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
+ .getOperatorDescriptor();
+ RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
+ writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
+ writer.open();
+ try {
+ treeIndexHelper.init();
+ ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
+ indexAccessor = treeIndex.createAccessor();
+ } catch (Exception e) {
+ // cleanup in case of failure
+ treeIndexHelper.deinit();
+ throw new HyracksDataException(e);
+ }
+ }
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- final ITreeIndex treeIndex = treeIndexOpHelper.getTreeIndex();
- accessor.reset(buffer);
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ try {
+ switch (op) {
+ case INSERT: {
+ indexAccessor.insert(tuple);
+ break;
+ }
+ case UPDATE: {
+ indexAccessor.update(tuple);
+ break;
+ }
+ case DELETE: {
+ indexAccessor.delete(tuple);
+ break;
+ }
+ default: {
+ throw new HyracksDataException("Unsupported operation " + op
+ + " in tree index InsertUpdateDelete operator");
+ }
+ }
+ } catch (HyracksDataException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ // Pass a copy of the frame to next op.
+ System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
+ FrameUtils.flushFrame(writeBuffer, writer);
+ }
- int tupleCount = accessor.getTupleCount();
- for (int i = 0; i < tupleCount; i++) {
- tuple.reset(accessor, i);
- try {
- switch (op) {
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ writer.close();
+ } finally {
+ treeIndexHelper.deinit();
+ }
+ }
- case INSERT: {
- treeIndex.insert(tuple, opCtx);
- break;
- }
-
- case UPDATE: {
- treeIndex.update(tuple, opCtx);
- break;
- }
-
- case DELETE: {
- treeIndex.delete(tuple, opCtx);
- break;
- }
-
- default: {
- throw new HyracksDataException("Unsupported operation "
- + op + " in tree index InsertUpdateDelete operator");
- }
-
- }
- } catch (HyracksDataException e) {
- throw e;
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- }
-
- // pass a copy of the frame to next op
- System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0,
- buffer.capacity());
- FrameUtils.flushFrame(writeBuffer, writer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- try {
- writer.close();
- } finally {
- treeIndexOpHelper.deinit();
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- writer.fail();
- }
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
deleted file mode 100644
index 6fea7e5..0000000
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public abstract class TreeIndexOpHelper {
-
- protected ITreeIndexFrame interiorFrame;
- protected ITreeIndexFrame leafFrame;
- protected MultiComparator cmp;
-
- protected ITreeIndex treeIndex;
- protected int indexFileId = -1;
- protected int partition;
-
- protected ITreeIndexOperatorDescriptorHelper opDesc;
- protected IHyracksTaskContext ctx;
-
- protected IndexHelperOpenMode mode;
-
- public TreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
- int partition, IndexHelperOpenMode mode) {
- this.opDesc = opDesc;
- this.ctx = ctx;
- this.mode = mode;
- this.partition = partition;
- }
-
- public void init() throws HyracksDataException {
- IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
- IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
- IFileSplitProvider fileSplitProvider = opDesc.getTreeIndexFileSplitProvider();
-
- FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
- boolean fileIsMapped = fileMapProvider.isMapped(f);
- if (!fileIsMapped) {
- bufferCache.createFile(f);
- }
- int fileId = fileMapProvider.lookupFileId(f);
- try {
- bufferCache.openFile(fileId);
- } catch (HyracksDataException e) {
- // Revert state of buffer cache since file failed to open.
- if (!fileIsMapped) {
- bufferCache.deleteFile(fileId);
- }
- throw e;
- }
-
- // Only set indexFileId member when openFile() succeeds,
- // otherwise deinit() will try to close the file that failed to open
- indexFileId = fileId;
-
- IndexRegistry<ITreeIndex> treeIndexRegistry = opDesc.getTreeIndexRegistryProvider().getRegistry(ctx);
- // Create new tree and register it.
- treeIndexRegistry.lock();
- try {
- // Check if tree has already been registered by another thread.
- treeIndex = treeIndexRegistry.get(indexFileId);
- if (treeIndex != null) {
- return;
- }
- IBinaryComparator[] comparators = new IBinaryComparator[opDesc.getTreeIndexComparatorFactories().length];
- for (int i = 0; i < opDesc.getTreeIndexComparatorFactories().length; i++) {
- comparators[i] = opDesc.getTreeIndexComparatorFactories()[i].createBinaryComparator();
- }
- cmp = new MultiComparator(comparators);
- treeIndex = createTreeIndex();
- if (mode == IndexHelperOpenMode.CREATE) {
- try {
- treeIndex.create(indexFileId);
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
- }
- treeIndex.open(indexFileId);
- treeIndexRegistry.register(indexFileId, treeIndex);
- } finally {
- treeIndexRegistry.unlock();
- }
- }
-
- // MUST be overridden
- public ITreeIndex createTreeIndex() throws HyracksDataException {
- throw new HyracksDataException("createTreeIndex Operation not implemented.");
- }
-
- // MUST be overridden
- public MultiComparator createMultiComparator(IBinaryComparator[] comparators) throws HyracksDataException {
- throw new HyracksDataException("createComparator Operation not implemented.");
- }
-
- public ITreeIndexCursor createDiskOrderScanCursor(ITreeIndexFrame leafFrame) throws HyracksDataException {
- return new TreeDiskOrderScanCursor(leafFrame);
- }
-
- public void deinit() throws HyracksDataException {
- if (indexFileId != -1) {
- IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
- bufferCache.closeFile(indexFileId);
- }
- }
-
- public ITreeIndex getTreeIndex() {
- return treeIndex;
- }
-
- public IHyracksTaskContext getHyracksTaskContext() {
- return ctx;
- }
-
- public ITreeIndexOperatorDescriptorHelper getOperatorDescriptor() {
- return opDesc;
- }
-
- public int getIndexFileId() {
- return indexFileId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index b14681a..96a2a94 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -1,38 +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.common.dataflow;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-public class TreeIndexStatsOperatorDescriptor extends
- AbstractTreeIndexOperatorDescriptor {
+public class TreeIndexStatsOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
+ private static final RecordDescriptor recDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
- public TreeIndexStatsOperatorDescriptor(JobSpecification spec,
- IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- IFileSplitProvider fileSplitProvider,
- ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories,
- ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 0, 0, null, storageManager, treeIndexRegistryProvider,
- fileSplitProvider, interiorFrameFactory, leafFrameFactory,
- typeTraits, comparatorFactories, opHelperFactory);
- }
+ public TreeIndexStatsOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider, IFileSplitProvider fileSplitProvider,
+ ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+ ITypeTrait[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
+ IIndexDataflowHelperFactory dataflowHelperFactory) {
+ super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+ leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
+ }
- @Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions) {
- return new TreeIndexStatsOperatorNodePushable(this, ctx, partition);
- }
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new TreeIndexStatsOperatorNodePushable(this, ctx, partition);
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index 82b1b38..ff007af 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -14,75 +14,76 @@
*/
package edu.uci.ics.hyracks.storage.am.common.dataflow;
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-public class TreeIndexStatsOperatorNodePushable extends
- AbstractOperatorNodePushable {
- private final TreeIndexOpHelper treeIndexOpHelper;
- private final IHyracksTaskContext ctx;
- private TreeIndexStatsGatherer statsGatherer;
+public class TreeIndexStatsOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+ private final TreeIndexDataflowHelper treeIndexHelper;
+ private final IHyracksTaskContext ctx;
+ private TreeIndexStatsGatherer statsGatherer;
- public TreeIndexStatsOperatorNodePushable(
- AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksTaskContext ctx, int partition) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory()
- .createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.CREATE);
- this.ctx = ctx;
- }
+ public TreeIndexStatsOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition) {
+ treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ opDesc, ctx, partition, false);
+ this.ctx = ctx;
+ }
- @Override
- public void deinitialize() throws HyracksDataException {
- }
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ }
- @Override
- public int getInputArity() {
- return 0;
- }
+ @Override
+ public IFrameWriter getInputFrameWriter(int index) {
+ return null;
+ }
- @Override
- public IFrameWriter getInputFrameWriter(int index) {
- return null;
- }
-
- @Override
- public void initialize() throws HyracksDataException {
- try {
- treeIndexOpHelper.init();
- treeIndexOpHelper.getTreeIndex().open(
- treeIndexOpHelper.getIndexFileId());
-
- ITreeIndex treeIndex = treeIndexOpHelper.getTreeIndex();
- IBufferCache bufferCache = treeIndexOpHelper
- .getOperatorDescriptor().getStorageManager()
- .getBufferCache(ctx);
-
- statsGatherer = new TreeIndexStatsGatherer(bufferCache,
- treeIndex.getFreePageManager(),
- treeIndexOpHelper.getIndexFileId(),
- treeIndex.getRootPageId());
- TreeIndexStats stats = statsGatherer.gatherStats(treeIndex
- .getLeafFrameFactory().createFrame(), treeIndex
- .getInteriorFrameFactory().createFrame(), treeIndex
- .getFreePageManager().getMetaDataFrameFactory()
- .createFrame());
- System.err.println(stats.toString());
- } catch (Exception e) {
- treeIndexOpHelper.deinit();
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- public void setOutputFrameWriter(int index, IFrameWriter writer,
- RecordDescriptor recordDesc) {
- }
+ @Override
+ public void initialize() throws HyracksDataException {
+ try {
+ writer.open();
+ treeIndexHelper.init();
+ ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
+ IBufferCache bufferCache = treeIndexHelper.getOperatorDescriptor().getStorageManager().getBufferCache(ctx);
+ statsGatherer = new TreeIndexStatsGatherer(bufferCache, treeIndex.getFreePageManager(),
+ treeIndexHelper.getIndexFileId(), treeIndex.getRootPageId());
+ TreeIndexStats stats = statsGatherer.gatherStats(treeIndex.getLeafFrameFactory().createFrame(), treeIndex
+ .getInteriorFrameFactory().createFrame(), treeIndex.getFreePageManager().getMetaDataFrameFactory()
+ .createFrame());
+ // Write the stats output as a single string field.
+ ByteBuffer frame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(frame, true);
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+ DataOutput dos = tb.getDataOutput();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(stats.toString(), dos);
+ tb.addFieldEndOffset();
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ FrameUtils.flushFrame(frame, writer);
+ } catch (Exception e) {
+ try {
+ treeIndexHelper.deinit();
+ } finally {
+ writer.fail();
+ }
+ } finally {
+ writer.close();
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
index 18d60ec..1a81c3f 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeDiskOrderScanCursor.java
@@ -31,12 +31,12 @@
private int fileId = -1;
private int currentPageId = -1;
private int maxPageId = -1;
- private ICachedPage page = null;
- private ITreeIndexFrame frame = null;
+ private ICachedPage page = null;
private IBufferCache bufferCache = null;
- private ITreeIndexTupleReference frameTuple;
-
+ private final ITreeIndexFrame frame;
+ private final ITreeIndexTupleReference frameTuple;
+
public TreeDiskOrderScanCursor(ITreeIndexFrame frame) {
this.frame = frame;
this.frameTuple = frame.createTupleReference();
@@ -145,4 +145,9 @@
public void setMaxPageId(int maxPageId) {
this.maxPageId = maxPageId;
}
+
+ @Override
+ public boolean exclusiveLatchNodes() {
+ return false;
+ }
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexUtils.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexUtils.java
new file mode 100644
index 0000000..389855f
--- /dev/null
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/util/IndexUtils.java
@@ -0,0 +1,29 @@
+/*
+ * 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.util;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+public class IndexUtils {
+ public static MultiComparator createMultiComparator(IBinaryComparatorFactory[] cmpFactories) {
+ IBinaryComparator[] cmps = new IBinaryComparator[cmpFactories.length];
+ for (int i = 0; i < cmpFactories.length; i++) {
+ cmps[i] = cmpFactories[i].createBinaryComparator();
+ }
+ return new MultiComparator(cmps);
+ }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
similarity index 64%
rename from hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java
rename to hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
index b3afe4a..499b60b 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptorHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexOperatorDescriptor.java
@@ -18,16 +18,12 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptor;
-public interface IInvertedIndexOperatorDescriptorHelper extends ITreeIndexOperatorDescriptorHelper {
- public IFileSplitProvider getInvIndexFileSplitProvider();
+public interface IInvertedIndexOperatorDescriptor extends ITreeIndexOperatorDescriptor {
+ public IBinaryComparatorFactory[] getInvListsComparatorFactories();
- public IBinaryComparatorFactory[] getInvIndexComparatorFactories();
-
- public ITypeTrait[] getInvIndexTypeTraits();
-
- public IIndexRegistryProvider<InvertedIndex> getInvIndexRegistryProvider();
+ public ITypeTrait[] getInvListsTypeTraits();
+
+ public IFileSplitProvider getInvListsFileSplitProvider();
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java
new file mode 100644
index 0000000..92770d6
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifierFactory.java
@@ -0,0 +1,22 @@
+/*
+ * 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.invertedindex.api;
+
+import java.io.Serializable;
+
+public interface IInvertedIndexSearchModifierFactory extends Serializable {
+ public IInvertedIndexSearchModifier createSearchModifier();
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
index b93f064..a8860e2 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
@@ -21,98 +21,100 @@
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.util.InvertedIndexUtils;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public abstract class AbstractInvertedIndexOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor
- implements IInvertedIndexOperatorDescriptorHelper {
+ implements IInvertedIndexOperatorDescriptor {
private static final long serialVersionUID = 1L;
- // general
+ // General.
protected final IStorageManagerInterface storageManager;
+ protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
- // btree
- protected final IFileSplitProvider btreeFileSplitProvider;
- protected final IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
- protected final ITreeIndexFrameFactory interiorFrameFactory;
- protected final ITreeIndexFrameFactory leafFrameFactory;
+ // Btree.
+ protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
+ protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
protected final ITypeTrait[] btreeTypeTraits;
protected final IBinaryComparatorFactory[] btreeComparatorFactories;
- protected final ITreeIndexOpHelperFactory opHelperFactory;
+ protected final IIndexDataflowHelperFactory btreeDataflowHelperFactory;
+ protected final IFileSplitProvider btreeFileSplitProvider;
- // inverted index
- protected final IFileSplitProvider invIndexFileSplitProvider;
- protected final IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider;
- protected final ITypeTrait[] invIndexTypeTraits;
- protected final IBinaryComparatorFactory[] invIndexComparatorFactories;
+ // Inverted index.
+ protected final ITypeTrait[] invListsTypeTraits;
+ protected final IBinaryComparatorFactory[] invListComparatorFactories;
+ protected final IFileSplitProvider invListsFileSplitProvider;
public AbstractInvertedIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
RecordDescriptor recDesc, IStorageManagerInterface storageManager,
- IFileSplitProvider btreeFileSplitProvider, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
- ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
- ITreeIndexOpHelperFactory opHelperFactory, IFileSplitProvider invIndexFileSplitProvider,
- IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
- IBinaryComparatorFactory[] invIndexComparatorFactories) {
+ IFileSplitProvider btreeFileSplitProvider, IFileSplitProvider invListsFileSplitProvider,
+ IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTrait[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTrait[] invListsTypeTraits,
+ IBinaryComparatorFactory[] invListComparatorFactories,
+ IIndexDataflowHelperFactory btreeDataflowHelperFactory) {
super(spec, inputArity, outputArity);
- // general
+ // General.
this.storageManager = storageManager;
+ this.indexRegistryProvider = indexRegistryProvider;
- // btree
+ // Btree.
+ this.btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
+ ITreeIndexTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
+ this.btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ this.btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ this.btreeComparatorFactories = tokenComparatorFactories;
+ this.btreeDataflowHelperFactory = btreeDataflowHelperFactory;
this.btreeFileSplitProvider = btreeFileSplitProvider;
- this.treeIndexRegistryProvider = treeIndexRegistryProvider;
- this.interiorFrameFactory = interiorFrameFactory;
- this.leafFrameFactory = leafFrameFactory;
- this.btreeTypeTraits = btreeTypeTraits;
- this.btreeComparatorFactories = btreeComparatorFactories;
- this.opHelperFactory = opHelperFactory;
- // inverted index
- this.invIndexFileSplitProvider = invIndexFileSplitProvider;
- this.invIndexRegistryProvider = invIndexRegistryProvider;
- this.invIndexTypeTraits = invIndexTypeTraits;
- this.invIndexComparatorFactories = invIndexComparatorFactories;
+ // Inverted index.
+ this.invListsTypeTraits = invListsTypeTraits;
+ this.invListComparatorFactories = invListComparatorFactories;
+ this.invListsFileSplitProvider = invListsFileSplitProvider;
- if (outputArity > 0)
+ if (outputArity > 0) {
recordDescriptors[0] = recDesc;
+ }
}
@Override
- public IFileSplitProvider getTreeIndexFileSplitProvider() {
+ public IFileSplitProvider getFileSplitProvider() {
return btreeFileSplitProvider;
}
+
+ @Override
+ public IFileSplitProvider getInvListsFileSplitProvider() {
+ return invListsFileSplitProvider;
+ }
@Override
public IBinaryComparatorFactory[] getTreeIndexComparatorFactories() {
return btreeComparatorFactories;
}
-
+
@Override
public ITypeTrait[] getTreeIndexTypeTraits() {
return btreeTypeTraits;
}
-
- @Override
- public int getTreeIndexFieldCount() {
- return btreeTypeTraits.length;
- }
@Override
public ITreeIndexFrameFactory getTreeIndexInteriorFactory() {
- return interiorFrameFactory;
+ return btreeInteriorFrameFactory;
}
@Override
public ITreeIndexFrameFactory getTreeIndexLeafFactory() {
- return leafFrameFactory;
+ return btreeLeafFrameFactory;
}
@Override
@@ -121,37 +123,27 @@
}
@Override
- public IIndexRegistryProvider<ITreeIndex> getTreeIndexRegistryProvider() {
- return treeIndexRegistryProvider;
- }
-
- @Override
public RecordDescriptor getRecordDescriptor() {
return recordDescriptors[0];
}
@Override
- public IIndexRegistryProvider<InvertedIndex> getInvIndexRegistryProvider() {
- return invIndexRegistryProvider;
+ public IBinaryComparatorFactory[] getInvListsComparatorFactories() {
+ return invListComparatorFactories;
}
@Override
- public IBinaryComparatorFactory[] getInvIndexComparatorFactories() {
- return invIndexComparatorFactories;
+ public ITypeTrait[] getInvListsTypeTraits() {
+ return invListsTypeTraits;
}
@Override
- public IFileSplitProvider getInvIndexFileSplitProvider() {
- return invIndexFileSplitProvider;
+ public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
+ return indexRegistryProvider;
}
-
+
@Override
- public ITypeTrait[] getInvIndexTypeTraits() {
- return invIndexTypeTraits;
- }
-
- @Override
- public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory() {
- return opHelperFactory;
+ public IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
+ return btreeDataflowHelperFactory;
}
}
\ No newline at end of file
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index b1cdefc..ffb94e7 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -29,19 +29,19 @@
private static final long serialVersionUID = 1L;
private final IBinaryTokenizerFactory tokenizerFactory;
- // fields that will be tokenized
+ // Fields that will be tokenized
private final int[] tokenFields;
- // operator will emit these projected fields for each token, e.g., as
+ // operator will append these key fields to each token, e.g., as
// payload for an inverted list
- // WARNING: too many projected fields can cause significant data blowup
- private final int[] projFields;
+ // WARNING: too many key fields can cause significant data blowup.
+ private final int[] keyFields;
public BinaryTokenizerOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
- IBinaryTokenizerFactory tokenizerFactory, int[] tokenFields, int[] projFields) {
+ IBinaryTokenizerFactory tokenizerFactory, int[] tokenFields, int[] keyFields) {
super(spec, 1, 1);
this.tokenizerFactory = tokenizerFactory;
this.tokenFields = tokenFields;
- this.projFields = projFields;
+ this.keyFields = keyFields;
recordDescriptors[0] = recDesc;
}
@@ -49,6 +49,6 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
int partition, int nPartitions) throws HyracksDataException {
return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(odId, 0),
- recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, projFields);
+ recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, keyFields);
}
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index 28e3db8..05f8b7d 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -22,12 +22,9 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
public class InvertedIndexBulkLoadOperatorDescriptor extends AbstractInvertedIndexOperatorDescriptor {
@@ -35,30 +32,22 @@
private static final long serialVersionUID = 1L;
private final int[] fieldPermutation;
- private final float btreeFillFactor;
- private final IInvertedListBuilder invListBuilder;
- public InvertedIndexBulkLoadOperatorDescriptor(JobSpecification spec, IStorageManagerInterface storageManager,
- int[] fieldPermutation, IFileSplitProvider btreeFileSplitProvider,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] btreeTypeTraits,
- IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
- ITreeIndexOpHelperFactory opHelperFactory, IFileSplitProvider invIndexFileSplitProvider,
- IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
- IBinaryComparatorFactory[] invIndexComparatorFactories, IInvertedListBuilder invListBuilder) {
- super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, treeIndexRegistryProvider,
- interiorFrameFactory, leafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeFillFactor,
- opHelperFactory, invIndexFileSplitProvider, invIndexRegistryProvider, invIndexTypeTraits,
- invIndexComparatorFactories);
+ public InvertedIndexBulkLoadOperatorDescriptor(JobSpecification spec, int[] fieldPermutation,
+ IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
+ IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+ ITypeTrait[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+ ITypeTrait[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+ IIndexDataflowHelperFactory btreeDataflowHelperFactory) {
+ super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
+ indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
+ invListComparatorFactories, btreeDataflowHelperFactory);
this.fieldPermutation = fieldPermutation;
- this.btreeFillFactor = btreeFillFactor;
- this.invListBuilder = invListBuilder;
}
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
- return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, btreeFillFactor,
- invListBuilder, recordDescProvider);
+ return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, recordDescProvider);
}
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
index 9d421b0..9dcabb4 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
@@ -22,18 +22,19 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
public class InvertedIndexBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
- private final TreeIndexOpHelper treeIndexOpHelper;
- private float btreeFillFactor;
-
- private final InvertedIndexOpHelper invIndexOpHelper;
- protected final IInvertedListBuilder invListBuilder;
+ private final TreeIndexDataflowHelper btreeDataflowHelper;
+ private final InvertedIndexDataflowHelper invIndexDataflowHelper;
+ private final IInvertedListBuilder invListBuilder;
+ private InvertedIndex invIndex;
private InvertedIndex.BulkLoadContext bulkLoadCtx;
private final IHyracksTaskContext ctx;
@@ -44,45 +45,49 @@
private IRecordDescriptorProvider recordDescProvider;
public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
- IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
- IInvertedListBuilder invListBuilder, IRecordDescriptorProvider recordDescProvider) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.CREATE);
- invIndexOpHelper = new InvertedIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
- this.btreeFillFactor = btreeFillFactor;
+ IHyracksTaskContext ctx, int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider) {
+ btreeDataflowHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
+ .createIndexDataflowHelper(opDesc, ctx, partition, true);
+ invIndexDataflowHelper = new InvertedIndexDataflowHelper(btreeDataflowHelper, opDesc, ctx, partition, true);
this.recordDescProvider = recordDescProvider;
this.ctx = ctx;
- this.invListBuilder = invListBuilder;
+ this.invListBuilder = new FixedSizeElementInvertedListBuilder(opDesc.getInvListsTypeTraits());
tuple.setFieldPermutation(fieldPermutation);
}
@Override
public void open() throws HyracksDataException {
- AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) treeIndexOpHelper
+ AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) btreeDataflowHelper
.getOperatorDescriptor();
RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), recDesc);
- // btree
+ // BTree.
try {
- treeIndexOpHelper.init();
- treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
+ btreeDataflowHelper.init();
} catch (Exception e) {
- // cleanup in case of failure
- treeIndexOpHelper.deinit();
- throw new HyracksDataException(e);
+ // Cleanup in case of failure.
+ btreeDataflowHelper.deinit();
+ if (e instanceof HyracksDataException) {
+ throw (HyracksDataException) e;
+ } else {
+ throw new HyracksDataException(e);
+ }
}
- // inverted index
+ // Inverted Index.
try {
- invIndexOpHelper.init();
- invIndexOpHelper.getInvIndex().open(invIndexOpHelper.getInvIndexFileId());
- bulkLoadCtx = invIndexOpHelper.getInvIndex().beginBulkLoad(invListBuilder, ctx.getFrameSize(),
- btreeFillFactor);
+ invIndexDataflowHelper.init();
+ invIndex = (InvertedIndex) invIndexDataflowHelper.getIndex();
+ bulkLoadCtx = invIndex.beginBulkLoad(invListBuilder, ctx.getFrameSize(), BTree.DEFAULT_FILL_FACTOR);
} catch (Exception e) {
- // cleanup in case of failure
- invIndexOpHelper.deinit();
- throw new HyracksDataException(e);
+ // Cleanup in case of failure.
+ invIndexDataflowHelper.deinit();
+ if (e instanceof HyracksDataException) {
+ throw (HyracksDataException) e;
+ } else {
+ throw new HyracksDataException(e);
+ }
}
}
@@ -92,16 +97,26 @@
int tupleCount = accessor.getTupleCount();
for (int i = 0; i < tupleCount; i++) {
tuple.reset(accessor, i);
- invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
+ try {
+ invIndex.bulkLoadAddTuple(bulkLoadCtx, tuple);
+ } catch (PageAllocationException e) {
+ throw new HyracksDataException(e);
+ }
}
}
@Override
public void close() throws HyracksDataException {
try {
- invIndexOpHelper.getInvIndex().endBulkLoad(bulkLoadCtx);
+ invIndex.endBulkLoad(bulkLoadCtx);
+ } catch (PageAllocationException e) {
+ throw new HyracksDataException(e);
} finally {
- treeIndexOpHelper.deinit();
+ try {
+ btreeDataflowHelper.deinit();
+ } finally {
+ invIndexDataflowHelper.deinit();
+ }
}
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
new file mode 100644
index 0000000..71717e4
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
@@ -0,0 +1,55 @@
+/*
+ * 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.invertedindex.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+
+public final class InvertedIndexDataflowHelper extends IndexDataflowHelper {
+ private final TreeIndexDataflowHelper btreeDataflowHelper;
+
+ public InvertedIndexDataflowHelper(TreeIndexDataflowHelper btreeDataflowHelper, IIndexOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition, boolean createIfNotExists) {
+ super(opDesc, ctx, partition, createIfNotExists);
+ this.btreeDataflowHelper = btreeDataflowHelper;
+ }
+
+ public FileReference getFilereference() {
+ AbstractInvertedIndexOperatorDescriptor invIndexOpDesc = (AbstractInvertedIndexOperatorDescriptor) opDesc;
+ IFileSplitProvider fileSplitProvider = invIndexOpDesc.getInvListsFileSplitProvider();
+ return fileSplitProvider.getFileSplits()[partition].getLocalFile();
+ }
+
+ @Override
+ public IIndex createIndexInstance() throws HyracksDataException {
+ IInvertedIndexOperatorDescriptor invIndexOpDesc = (IInvertedIndexOperatorDescriptor) opDesc;
+ MultiComparator cmp = IndexUtils.createMultiComparator(invIndexOpDesc.getInvListsComparatorFactories());
+ // Assumes btreeDataflowHelper.init() has already been called.
+ BTree btree = (BTree) btreeDataflowHelper.getIndex();
+ return new InvertedIndex(opDesc.getStorageManager().getBufferCache(ctx), btree,
+ invIndexOpDesc.getInvListsTypeTraits(), cmp);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
deleted file mode 100644
index 684bb20..0000000
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * 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.invertedindex.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public final class InvertedIndexOpHelper {
-
- private InvertedIndex invIndex;
- private int invIndexFileId = -1;
- private int partition;
-
- private IInvertedIndexOperatorDescriptorHelper opDesc;
- private IHyracksTaskContext ctx;
-
- private IndexHelperOpenMode mode;
-
- public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
- int partition, IndexHelperOpenMode mode) {
- this.opDesc = opDesc;
- this.ctx = ctx;
- this.mode = mode;
- this.partition = partition;
- }
-
- public void init() throws HyracksDataException {
- IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
- IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
- IFileSplitProvider fileSplitProvider = opDesc.getInvIndexFileSplitProvider();
-
- FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
- boolean fileIsMapped = fileMapProvider.isMapped(f);
-
- switch (mode) {
-
- case OPEN: {
- if (!fileIsMapped) {
- throw new HyracksDataException("Trying to open inverted index from unmapped file " + f.toString());
- }
- }
- break;
-
- case CREATE:
- case ENLIST: {
- if (!fileIsMapped) {
- bufferCache.createFile(f);
- }
- }
- break;
-
- }
-
- int fileId = fileMapProvider.lookupFileId(f);
- try {
- bufferCache.openFile(fileId);
- } catch (HyracksDataException e) {
- // revert state of buffer cache since file failed to open
- if (!fileIsMapped) {
- bufferCache.deleteFile(fileId);
- }
- throw e;
- }
-
- // only set btreeFileId member when openFile() succeeds,
- // otherwise deinit() will try to close the file that failed to open
- invIndexFileId = fileId;
- IndexRegistry<InvertedIndex> invIndexRegistry = opDesc.getInvIndexRegistryProvider().getRegistry(ctx);
- invIndex = invIndexRegistry.get(invIndexFileId);
- if (invIndex == null) {
-
- // create new inverted index and register it
- invIndexRegistry.lock();
- try {
- // check if inverted index has already been registered by
- // another thread
- invIndex = invIndexRegistry.get(invIndexFileId);
- if (invIndex == null) {
- // this thread should create and register the inverted index
-
- IBinaryComparator[] comparators = new IBinaryComparator[opDesc.getInvIndexComparatorFactories().length];
- for (int i = 0; i < opDesc.getInvIndexComparatorFactories().length; i++) {
- comparators[i] = opDesc.getInvIndexComparatorFactories()[i].createBinaryComparator();
- }
-
- MultiComparator cmp = new MultiComparator(comparators);
-
- // assumes btree has already been registered
- IFileSplitProvider btreeFileSplitProvider = opDesc.getTreeIndexFileSplitProvider();
- IndexRegistry<ITreeIndex> treeIndexRegistry = opDesc.getTreeIndexRegistryProvider()
- .getRegistry(ctx);
- FileReference btreeFile = btreeFileSplitProvider.getFileSplits()[partition].getLocalFile();
- boolean btreeFileIsMapped = fileMapProvider.isMapped(btreeFile);
- if (!btreeFileIsMapped) {
- throw new HyracksDataException(
- "Trying to create inverted index, but associated BTree file has not been mapped");
- }
- int btreeFileId = fileMapProvider.lookupFileId(f);
- BTree btree = (BTree) treeIndexRegistry.get(btreeFileId);
-
- invIndex = new InvertedIndex(bufferCache, btree, opDesc.getInvIndexTypeTraits(), cmp);
- invIndex.open(invIndexFileId);
- invIndexRegistry.register(invIndexFileId, invIndex);
- }
- } finally {
- invIndexRegistry.unlock();
- }
- }
- }
-
- public void deinit() throws HyracksDataException {
- if (invIndexFileId != -1) {
- IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
- bufferCache.closeFile(invIndexFileId);
- }
- }
-
- public InvertedIndex getInvIndex() {
- return invIndex;
- }
-
- public ITreeIndexOperatorDescriptorHelper getOperatorDescriptor() {
- return opDesc;
- }
-
- public int getInvIndexFileId() {
- return invIndexFileId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
new file mode 100644
index 0000000..f1704cc
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * 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.invertedindex.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+
+public class InvertedIndexSearchOperatorDescriptor extends AbstractInvertedIndexOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final int queryField;
+ private final IBinaryTokenizerFactory queryTokenizerFactory;
+ private final IInvertedIndexSearchModifierFactory searchModifierFactory;
+
+ public InvertedIndexSearchOperatorDescriptor(JobSpecification spec,
+ int queryField, IStorageManagerInterface storageManager, IFileSplitProvider btreeFileSplitProvider,
+ IFileSplitProvider invListsFileSplitProvider, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+ ITypeTrait[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
+ ITypeTrait[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
+ IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
+ IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc) {
+ super(spec, 1, 1, recDesc, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
+ indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
+ invListComparatorFactories, btreeDataflowHelperFactory);
+ this.queryField = queryField;
+ this.queryTokenizerFactory = queryTokenizerFactory;
+ this.searchModifierFactory = searchModifierFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ IBinaryTokenizer tokenizer = queryTokenizerFactory.createTokenizer();
+ IInvertedIndexSearchModifier searchModifier = searchModifierFactory.createSearchModifier();
+ return new InvertedIndexSearchOperatorNodePushable(this, ctx, partition, queryField, searchModifier, tokenizer,
+ recordDescProvider);
+ }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
new file mode 100644
index 0000000..ceaff1e
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
@@ -0,0 +1,183 @@
+/*
+ * 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.invertedindex.dataflow;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexResultCursor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.OccurrenceThresholdPanicException;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.SearchResultCursor;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.TOccurrenceSearcher;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
+
+public class InvertedIndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+ private final TreeIndexDataflowHelper btreeDataflowHelper;
+ private final InvertedIndexDataflowHelper invIndexDataflowHelper;
+ private final IHyracksTaskContext ctx;
+ private final int queryField;
+ private FrameTupleAccessor accessor;
+ private FrameTupleReference tuple;
+ private IRecordDescriptorProvider recordDescProvider;
+ private InvertedIndex invIndex;
+
+ private final IInvertedIndexSearchModifier searchModifier;
+ private final IBinaryTokenizer queryTokenizer;
+ private TOccurrenceSearcher searcher;
+ private IInvertedIndexResultCursor resultCursor;
+
+ private ByteBuffer writeBuffer;
+ private FrameTupleAppender appender;
+ private ArrayTupleBuilder tb;
+ private DataOutput dos;
+
+ private final AbstractInvertedIndexOperatorDescriptor opDesc;
+
+ public InvertedIndexSearchOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
+ IHyracksTaskContext ctx, int partition, int queryField, IInvertedIndexSearchModifier searchModifier,
+ IBinaryTokenizer queryTokenizer, IRecordDescriptorProvider recordDescProvider) {
+ this.opDesc = opDesc;
+ btreeDataflowHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory()
+ .createIndexDataflowHelper(opDesc, ctx, partition, false);
+ invIndexDataflowHelper = new InvertedIndexDataflowHelper(btreeDataflowHelper, opDesc, ctx, partition, false);
+ this.ctx = ctx;
+ this.queryField = queryField;
+ this.searchModifier = searchModifier;
+ this.queryTokenizer = queryTokenizer;
+ this.recordDescProvider = recordDescProvider;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ tuple = new FrameTupleReference();
+ // BTree.
+ try {
+ btreeDataflowHelper.init();
+ } catch (Exception e) {
+ // Cleanup in case of failure/
+ btreeDataflowHelper.deinit();
+ if (e instanceof HyracksDataException) {
+ throw (HyracksDataException) e;
+ } else {
+ throw new HyracksDataException(e);
+ }
+ }
+ // Inverted Index.
+ try {
+ invIndexDataflowHelper.init();
+ invIndex = (InvertedIndex) invIndexDataflowHelper.getIndex();
+ } catch (Exception e) {
+ // Cleanup in case of failure.
+ invIndexDataflowHelper.deinit();
+ if (e instanceof HyracksDataException) {
+ throw (HyracksDataException) e;
+ } else {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ writeBuffer = btreeDataflowHelper.getHyracksTaskContext().allocateFrame();
+ tb = new ArrayTupleBuilder(opDesc.getInvListsTypeTraits().length);
+ dos = tb.getDataOutput();
+ appender = new FrameTupleAppender(btreeDataflowHelper.getHyracksTaskContext().getFrameSize());
+ appender.reset(writeBuffer, true);
+
+ searcher = new TOccurrenceSearcher(ctx, invIndex, queryTokenizer);
+ resultCursor = new SearchResultCursor(searcher.createResultFrameTupleAccessor(),
+ searcher.createResultTupleReference());
+
+ writer.open();
+ }
+
+ private void writeSearchResults() throws Exception {
+ while (resultCursor.hasNext()) {
+ resultCursor.next();
+ tb.reset();
+ ITupleReference invListElement = resultCursor.getTuple();
+ int invListFields = opDesc.getInvListsTypeTraits().length;
+ for (int i = 0; i < invListFields; i++) {
+ dos.write(invListElement.getFieldData(i), invListElement.getFieldStart(i),
+ invListElement.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ appender.reset(writeBuffer, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ try {
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ searcher.reset();
+ try {
+ searcher.reset();
+ searcher.search(resultCursor, tuple, queryField, searchModifier);
+ writeSearchResults();
+ } catch (OccurrenceThresholdPanicException e) {
+ // Ignore panic cases for now.
+ }
+ }
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ }
+ writer.close();
+ } finally {
+ try {
+ btreeDataflowHelper.deinit();
+ } finally {
+ invIndexDataflowHelper.deinit();
+ }
+ }
+ }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 75fdf2a..1118d04 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -29,11 +29,13 @@
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
@@ -49,36 +51,41 @@
* implemented features: updates (insert/update/delete) Limitations: a query
* cannot exceed the size of a Hyracks frame
*/
-public class InvertedIndex {
+public class InvertedIndex implements IIndex {
private BTree btree;
private int rootPageId = 0;
private IBufferCache bufferCache;
private int fileId;
- private final ITypeTrait[] typeTraits;
+ private final ITypeTrait[] invListTypeTraits;
private final MultiComparator invListCmp;
private final int numTokenFields;
private final int numInvListKeys;
- public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTrait[] typeTraits, MultiComparator invListCmp) {
+ public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTrait[] invListTypeTraits, MultiComparator invListCmp) {
this.bufferCache = bufferCache;
this.btree = btree;
this.invListCmp = invListCmp;
- this.typeTraits = typeTraits;
+ this.invListTypeTraits = invListTypeTraits;
this.numTokenFields = btree.getMultiComparator().getKeyFieldCount();
this.numInvListKeys = invListCmp.getKeyFieldCount();
}
+ @Override
public void open(int fileId) {
this.fileId = fileId;
}
+ @Override
+ public void create(int indexFileId) throws HyracksDataException {
+ }
+
public void close() {
this.fileId = -1;
}
public BulkLoadContext beginBulkLoad(IInvertedListBuilder invListBuilder, int hyracksFrameSize,
- float btreeFillFactor) throws HyracksDataException, TreeIndexException {
+ float btreeFillFactor) throws HyracksDataException, TreeIndexException, PageAllocationException {
BulkLoadContext ctx = new BulkLoadContext(invListBuilder, hyracksFrameSize, btreeFillFactor);
ctx.init(rootPageId, fileId);
return ctx;
@@ -90,7 +97,7 @@
// the next invListCmp.getKeyFieldCount() fields in tuple are keys of the
// inverted list (e.g., primary key)
// key fields of inverted list are fixed size
- public void bulkLoadAddTuple(BulkLoadContext ctx, ITupleReference tuple) throws HyracksDataException {
+ public void bulkLoadAddTuple(BulkLoadContext ctx, ITupleReference tuple) throws HyracksDataException, PageAllocationException {
// first inverted list, copy token to baaos and start new list
if (ctx.currentInvListTokenBaaos.size() == 0) {
@@ -151,9 +158,9 @@
}
}
- public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, BTreeOpContext btreeOpCtx,
+ public boolean openCursor(ITreeIndexCursor btreeCursor, RangePredicate btreePred, ITreeIndexAccessor btreeAccessor,
IInvertedListCursor invListCursor) throws Exception {
- btree.search(btreeCursor, btreePred, btreeOpCtx);
+ btreeAccessor.search(btreeCursor, btreePred);
boolean ret = false;
if (btreeCursor.hasNext()) {
@@ -183,7 +190,7 @@
return ret;
}
- public void createAndInsertBTreeTuple(BulkLoadContext ctx) throws HyracksDataException {
+ public void createAndInsertBTreeTuple(BulkLoadContext ctx) throws HyracksDataException, PageAllocationException {
// build tuple
ctx.btreeTupleBuilder.reset();
ctx.btreeTupleBuilder.addField(ctx.currentInvListTokenBaaos.getByteArray(), 0,
@@ -204,7 +211,7 @@
btree.bulkLoadAddTuple(ctx.btreeFrameTupleReference, ctx.btreeBulkLoadCtx);
}
- public void endBulkLoad(BulkLoadContext ctx) throws HyracksDataException {
+ public void endBulkLoad(BulkLoadContext ctx) throws HyracksDataException, PageAllocationException {
// create entry in btree for last inverted list
createAndInsertBTreeTuple(ctx);
btree.endBulkLoad(ctx.btreeBulkLoadCtx);
@@ -224,7 +231,7 @@
}
public ITypeTrait[] getTypeTraits() {
- return typeTraits;
+ return invListTypeTraits;
}
public BTree getBTree() {
@@ -243,7 +250,7 @@
private int currentInvListStartPageId;
private int currentInvListStartOffset;
private final ByteArrayAccessibleOutputStream currentInvListTokenBaaos = new ByteArrayAccessibleOutputStream();
- private final FixedSizeTupleReference currentInvListToken = new FixedSizeTupleReference(typeTraits);
+ private final FixedSizeTupleReference currentInvListToken = new FixedSizeTupleReference(invListTypeTraits);
private int currentPageId;
private ICachedPage currentPage;
@@ -267,7 +274,7 @@
this.btreeFillFactor = btreeFillFactor;
}
- public void init(int startPageId, int fileId) throws HyracksDataException, TreeIndexException {
+ public void init(int startPageId, int fileId) throws HyracksDataException, TreeIndexException, PageAllocationException {
btreeBulkLoadCtx = btree.beginBulkLoad(BTree.DEFAULT_FILL_FACTOR);
currentPageId = startPageId;
currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
@@ -290,5 +297,5 @@
currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
currentPage.acquireWriteLatch();
}
- };
+ }
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index 0f88c0b..dd09ef7 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -36,12 +36,11 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexResultCursor;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
@@ -69,7 +68,7 @@
protected final ITreeIndexCursor btreeCursor;
protected final FrameTupleReference searchKey = new FrameTupleReference();
protected final RangePredicate btreePred = new RangePredicate(true, null, null, true, true, null, null);
- protected final BTreeOpContext btreeOpCtx;
+ protected final ITreeIndexAccessor btreeAccessor;
protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
@@ -95,7 +94,7 @@
leafFrame = invIndex.getBTree().getLeafFrameFactory().createFrame();
interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
- btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+ btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
ITypeTrait[] invListFields = invIndex.getTypeTraits();
invListFieldsWithCount = new TypeTrait[invListFields.length + 1];
int tmp = 0;
@@ -107,8 +106,6 @@
invListFieldsWithCount[invListFields.length] = new TypeTrait(4);
invListKeyLength = tmp;
- btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH);
-
resultFrameTupleApp = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), invListFieldsWithCount);
resultFrameTupleAcc = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), invListFieldsWithCount);
resultTuple = new FixedSizeTupleReference(invListFieldsWithCount);
@@ -129,7 +126,8 @@
queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
queryTokenFrame = ctx.allocateFrame();
-
+
+ btreeAccessor = invIndex.getBTree().createAccessor();
currentNumResults = 0;
}
@@ -143,12 +141,12 @@
currentNumResults = 0;
}
- public void search(IInvertedIndexResultCursor resultCursor, ITupleReference queryTuple, int queryFieldIndex,
+ public void search(IInvertedIndexResultCursor resultCursor, ITupleReference queryTuple, int queryField,
IInvertedIndexSearchModifier searchModifier) throws Exception {
queryTokenAppender.reset(queryTokenFrame, true);
- queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
- queryTuple.getFieldLength(queryFieldIndex));
+ queryTokenizer.reset(queryTuple.getFieldData(queryField), queryTuple.getFieldStart(queryField),
+ queryTuple.getFieldLength(queryField));
while (queryTokenizer.hasNext()) {
queryTokenizer.next();
@@ -181,7 +179,7 @@
invListCursors.clear();
for (int i = 0; i < numQueryTokens; i++) {
searchKey.reset(queryTokenAccessor, i);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursorCache.get(i));
+ invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursorCache.get(i));
invListCursors.add(invListCursorCache.get(i));
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java
new file mode 100644
index 0000000..0db6008
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/ConjunctiveSearchModifierFactory.java
@@ -0,0 +1,28 @@
+/*
+ * 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.invertedindex.searchmodifiers;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class ConjunctiveSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IInvertedIndexSearchModifier createSearchModifier() {
+ return new ConjunctiveSearchModifier();
+ }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java
new file mode 100644
index 0000000..128d9db
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/EditDistanceSearchModifierFactory.java
@@ -0,0 +1,37 @@
+/*
+ * 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.invertedindex.searchmodifiers;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class EditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int gramLength;
+ private final int edThresh;
+
+ public EditDistanceSearchModifierFactory(int gramLength, int edThresh) {
+ this.gramLength = gramLength;
+ this.edThresh = edThresh;
+ }
+
+ @Override
+ public IInvertedIndexSearchModifier createSearchModifier() {
+ return new EditDistanceSearchModifier(gramLength, edThresh);
+ }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java
new file mode 100644
index 0000000..bd27c03
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchmodifiers/JaccardSearchModifierFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.invertedindex.searchmodifiers;
+
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+
+public class JaccardSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final float jaccThresh;
+
+ public JaccardSearchModifierFactory(float jaccThresh) {
+ this.jaccThresh = jaccThresh;
+ }
+
+ @Override
+ public IInvertedIndexSearchModifier createSearchModifier() {
+ return new JaccardSearchModifier(jaccThresh);
+ }
+}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
new file mode 100644
index 0000000..5a62f51
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
@@ -0,0 +1,47 @@
+/*
+ * 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.invertedindex.util;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+
+public class InvertedIndexUtils {
+ // Type traits to be appended to the token type trait which finally form the BTree field type traits.
+ private static final ITypeTrait[] btreeValueTypeTraits = new ITypeTrait[4];
+ static {
+ // startPageId
+ btreeValueTypeTraits[0] = new TypeTrait(4);
+ // endPageId
+ btreeValueTypeTraits[1] = new TypeTrait(4);
+ // startOff
+ btreeValueTypeTraits[2] = new TypeTrait(4);
+ // numElements
+ btreeValueTypeTraits[3] = new TypeTrait(4);
+ }
+
+ public static ITypeTrait[] getBTreeTypeTraits(ITypeTrait[] tokenTypeTraits) {
+ ITypeTrait[] btreeTypeTraits = new ITypeTrait[tokenTypeTraits.length + btreeValueTypeTraits.length];
+ // Set key type traits.
+ for (int i = 0; i < tokenTypeTraits.length; i++) {
+ btreeTypeTraits[i] = tokenTypeTraits[i];
+ }
+ // Set value type traits.
+ for (int i = 0; i < btreeValueTypeTraits.length; i++) {
+ btreeTypeTraits[i + tokenTypeTraits.length] = btreeValueTypeTraits[i];
+ }
+ return btreeTypeTraits;
+ }
+}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java
new file mode 100644
index 0000000..51abb79
--- /dev/null
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelper.java
@@ -0,0 +1,49 @@
+/*
+ * 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.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+
+public class RTreeDataflowHelper extends TreeIndexDataflowHelper {
+
+ public RTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
+ boolean createIfNotExists) {
+ super(opDesc, ctx, partition, createIfNotExists);
+ }
+
+ @Override
+ public ITreeIndex createIndexInstance() throws HyracksDataException {
+ MultiComparator cmp = IndexUtils.createMultiComparator(treeOpDesc.getTreeIndexComparatorFactories());
+ IBufferCache bufferCache = treeOpDesc.getStorageManager().getBufferCache(ctx);
+ ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0,
+ metaDataFrameFactory);
+ return new RTree(bufferCache, treeOpDesc.getTreeIndexTypeTraits().length, cmp, freePageManager,
+ treeOpDesc.getTreeIndexInteriorFactory(), treeOpDesc.getTreeIndexLeafFactory());
+ }
+}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java
new file mode 100644
index 0000000..8cd4e95
--- /dev/null
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeDataflowHelperFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.rtree.dataflow;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
+
+public class RTreeDataflowHelperFactory implements IIndexDataflowHelperFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IndexDataflowHelper createIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, boolean createIfNotExists) {
+ return new RTreeDataflowHelper(opDesc, ctx, partition, createIfNotExists);
+ }
+}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
deleted file mode 100644
index 8c83636..0000000
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.rtree.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-
-public class RTreeOpHelper extends TreeIndexOpHelper {
-
- public RTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
- IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode) {
- super(opDesc, ctx, partition, mode);
- }
-
- public ITreeIndex createTreeIndex() throws HyracksDataException {
- IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(
- ctx);
- ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, indexFileId, 0, metaDataFrameFactory);
-
- return new RTree(bufferCache, opDesc.getTreeIndexFieldCount(), cmp,
- freePageManager, opDesc.getTreeIndexInteriorFactory(),
- opDesc.getTreeIndexLeafFactory());
- }
-}
\ No newline at end of file
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
deleted file mode 100644
index 6afc37e..0000000
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.rtree.dataflow;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-
-public class RTreeOpHelperFactory implements ITreeIndexOpHelperFactory {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public TreeIndexOpHelper createTreeIndexOpHelper(
- ITreeIndexOperatorDescriptorHelper opDesc, IHyracksTaskContext ctx,
- int partition, IndexHelperOpenMode mode) {
- return new RTreeOpHelper(opDesc, ctx, partition, mode);
- }
-}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 407ebc9..586324f 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -23,40 +23,33 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-public class RTreeSearchOperatorDescriptor extends
- AbstractTreeIndexOperatorDescriptor {
+public class RTreeSearchOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- private int[] keyFields; // fields in input tuple to be used as keys
+ private int[] keyFields; // fields in input tuple to be used as keys
- public RTreeSearchOperatorDescriptor(JobSpecification spec,
- RecordDescriptor recDesc, IStorageManagerInterface storageManager,
- IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- IFileSplitProvider fileSplitProvider,
- ITreeIndexFrameFactory interiorFrameFactory,
- ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
- IBinaryComparatorFactory[] comparatorFactories,
- int[] keyFields, ITreeIndexOpHelperFactory opHelperFactory) {
- super(spec, 1, 1, recDesc, storageManager, treeIndexRegistryProvider,
- fileSplitProvider, interiorFrameFactory, leafFrameFactory,
- typeTraits, comparatorFactories,
- opHelperFactory);
- this.keyFields = keyFields;
- }
+ public RTreeSearchOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
+ IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
+ IFileSplitProvider fileSplitProvider, ITreeIndexFrameFactory interiorFrameFactory,
+ ITreeIndexFrameFactory leafFrameFactory, ITypeTrait[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+ IIndexDataflowHelperFactory dataflowHelperFactory) {
+ super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, interiorFrameFactory,
+ leafFrameFactory, typeTraits, comparatorFactories, dataflowHelperFactory);
+ this.keyFields = keyFields;
+ }
- @Override
- public IOperatorNodePushable createPushRuntime(
- final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions) {
- return new RTreeSearchOperatorNodePushable(this, ctx, partition,
- recordDescProvider, keyFields);
- }
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ return new RTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields);
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 4142ebf..9f62b8f 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -29,23 +29,21 @@
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
public class RTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private TreeIndexOpHelper treeIndexOpHelper;
+ private TreeIndexDataflowHelper treeIndexHelper;
private FrameTupleAccessor accessor;
private ByteBuffer writeBuffer;
@@ -60,15 +58,14 @@
private ITreeIndexCursor cursor;
private ITreeIndexFrame interiorFrame;
private ITreeIndexFrame leafFrame;
- private RTreeOpContext opCtx;
+ private ITreeIndexAccessor indexAccessor;
private RecordDescriptor recDesc;
public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
- IndexHelperOpenMode.OPEN);
-
+ treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
+ opDesc, ctx, partition, false);
this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
if (keyFields != null && keyFields.length > 0) {
searchKey = new PermutingFrameTupleReference();
@@ -78,46 +75,37 @@
@Override
public void open() throws HyracksDataException {
- AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
+ AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
.getOperatorDescriptor();
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
-
+ accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
interiorFrame = opDesc.getTreeIndexInteriorFactory().createFrame();
leafFrame = opDesc.getTreeIndexLeafFactory().createFrame();
cursor = new RTreeSearchCursor((IRTreeInteriorFrame) interiorFrame, (IRTreeLeafFrame) leafFrame);
-
try {
-
- treeIndexOpHelper.init();
+ treeIndexHelper.init();
writer.open();
try {
- rtree = (RTree) treeIndexOpHelper.getTreeIndex();
-
+ rtree = (RTree) treeIndexHelper.getIndex();
int keySearchFields = rtree.getCmp().getComparators().length;
-
IBinaryComparator[] keySearchComparators = new IBinaryComparator[keySearchFields];
for (int i = 0; i < keySearchFields; i++) {
keySearchComparators[i] = rtree.getCmp().getComparators()[i];
}
-
cmp = new MultiComparator(keySearchComparators);
-
searchPred = new SearchPredicate(searchKey, cmp);
-
- writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+ writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
tb = new ArrayTupleBuilder(rtree.getFieldCount());
dos = tb.getDataOutput();
- appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
+ appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
-
- opCtx = rtree.createOpContext(IndexOp.SEARCH);
+ indexAccessor = rtree.createAccessor();
} catch (Exception e) {
writer.fail();
throw e;
}
} catch (Exception e) {
- treeIndexOpHelper.deinit();
+ treeIndexHelper.deinit();
throw new HyracksDataException(e);
}
}
@@ -154,7 +142,7 @@
searchPred.setSearchKey(searchKey);
cursor.reset();
- rtree.search(cursor, searchPred, opCtx);
+ indexAccessor.search(cursor, searchPred);
writeSearchResults();
}
} catch (Exception e) {
@@ -175,7 +163,7 @@
throw new HyracksDataException(e);
}
} finally {
- treeIndexOpHelper.deinit();
+ treeIndexHelper.deinit();
}
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
index 0486496..546a4d4 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/frames/RTreeNSMFrame.java
@@ -35,7 +35,7 @@
public abstract class RTreeNSMFrame extends TreeIndexNSMFrame implements
IRTreeFrame {
protected static final int pageNsnOff = smFlagOff + 1;
- protected static final int rightPageOff = pageNsnOff + 4;
+ protected static final int rightPageOff = pageNsnOff + 8;
protected ITreeIndexTupleReference[] tuples;
protected ITreeIndexTupleReference cmpFrameTuple;
@@ -86,7 +86,7 @@
@Override
public void initBuffer(byte level) {
super.initBuffer(level);
- buf.putInt(pageNsnOff, 0);
+ buf.putLong(pageNsnOff, 0);
buf.putInt(rightPageOff, -1);
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
index cb553f7..5f78dcc 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java
@@ -26,12 +26,15 @@
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
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.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.IndexType;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.FrameOpSpaceStatus;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
@@ -84,7 +87,7 @@
this.cmp = cmp;
this.freePageManager = freePageManager;
this.interiorFrameFactory = interiorFrameFactory;
- this.leafFrameFactory = leafFrameFactory;
+ this.leafFrameFactory = leafFrameFactory;
globalNsn = new AtomicLong();
this.treeLatch = new ReentrantReadWriteLock(true);
this.diskOrderScanPredicate = new SearchPredicate(null, cmp);
@@ -242,16 +245,13 @@
fileId = -1;
}
- @Override
- public RTreeOpContext createOpContext(IndexOp op) {
- return new RTreeOpContext(op,
- (IRTreeLeafFrame) leafFrameFactory.createFrame(),
- (IRTreeInteriorFrame) interiorFrameFactory.createFrame(),
- freePageManager.getMetaDataFrameFactory().createFrame(), 8);
- }
+ private RTreeOpContext createOpContext() {
+ return new RTreeOpContext((IRTreeLeafFrame) leafFrameFactory.createFrame(),
+ (IRTreeInteriorFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
+ .createFrame(), 8);
+ }
- @Override
- public void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
+ private void insert(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException, PageAllocationException {
RTreeOpContext ctx = (RTreeOpContext) ictx;
ctx.reset();
ctx.setTuple(tuple);
@@ -292,115 +292,142 @@
public ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException {
int pageId = rootPage;
boolean writeLatched = false;
+ boolean readLatched = false;
+ boolean succeed = false;
ICachedPage node = null;
boolean isLeaf = false;
long pageLsn = 0, parentLsn = 0;
- while (true) {
- if (!writeLatched) {
- node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
- incrementPins();
- ctx.interiorFrame.setPage(node);
- isLeaf = ctx.interiorFrame.isLeaf();
- if (isLeaf) {
- node.acquireWriteLatch();
- incrementWriteLatchesAcquired();
- writeLatched = true;
+ try {
- if (!ctx.interiorFrame.isLeaf()) {
+ while (true) {
+ if (!writeLatched) {
+ node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ incrementPins();
+ ctx.interiorFrame.setPage(node);
+ isLeaf = ctx.interiorFrame.isLeaf();
+ if (isLeaf) {
+ node.acquireWriteLatch();
+ writeLatched = true;
+ incrementWriteLatchesAcquired();
+
+ if (!ctx.interiorFrame.isLeaf()) {
+ node.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
+ continue;
+ }
+ } else {
+ // Be optimistic and grab read latch first. We will swap
+ // it
+ // to write latch if we need to enlarge the best child
+ // tuple.
+ node.acquireReadLatch();
+ readLatched = true;
+ incrementReadLatchesAcquired();
+ }
+ }
+
+ if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
+ // Concurrent split detected, go back to parent and
+ // re-choose
+ // the best child
+ if (writeLatched) {
node.releaseWriteLatch();
+ writeLatched = false;
incrementWriteLatchesReleased();
bufferCache.unpin(node);
incrementUnpins();
- writeLatched = false;
- continue;
+ } else {
+ node.releaseReadLatch();
+ readLatched = false;
+ incrementReadLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
}
- } else {
- // Be optimistic and grab read latch first. We will swap it
- // to write latch if we need to enlarge the best child
- // tuple.
- node.acquireReadLatch();
- incrementReadLatchesAcquired();
- }
- }
- if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
- // Concurrent split detected, go back to parent and re-choose
- // the best child
- if (writeLatched) {
+ pageId = ctx.pathList.getLastPageId();
+ if (pageId != rootPage) {
+ parentLsn = ctx.pathList.getPageLsn(ctx.pathList.size() - 2);
+ }
+ ctx.pathList.moveLast();
+ continue;
+ }
+
+ pageLsn = ctx.interiorFrame.getPageLsn();
+ ctx.pathList.add(pageId, pageLsn, -1);
+
+ if (!isLeaf) {
+ // findBestChild must be called *before* getBestChildPageId
+ ctx.interiorFrame.findBestChild(ctx.getTuple(), cmp);
+ int childPageId = ctx.interiorFrame.getBestChildPageId();
+
+ if (!writeLatched) {
+ node.releaseReadLatch();
+ readLatched = false;
+ incrementReadLatchesReleased();
+ // TODO: do we need to un-pin and pin again?
+ bufferCache.unpin(node);
+ incrementUnpins();
+
+ node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ incrementPins();
+ node.acquireWriteLatch();
+ writeLatched = true;
+ incrementWriteLatchesAcquired();
+ ctx.interiorFrame.setPage(node);
+
+ if (ctx.interiorFrame.getPageLsn() != pageLsn) {
+ // The page was changed while we unlocked it; thus,
+ // retry (re-choose best child)
+
+ ctx.pathList.moveLast();
+ continue;
+ }
+ }
+
+ // We don't need to reset the frameTuple because it is
+ // already pointing to the best child
+ ctx.interiorFrame.enlarge(ctx.getTuple(), cmp);
+
node.releaseWriteLatch();
+ writeLatched = false;
incrementWriteLatchesReleased();
bufferCache.unpin(node);
incrementUnpins();
- writeLatched = false;
+
+ pageId = childPageId;
+ parentLsn = pageLsn;
} else {
- node.releaseReadLatch();
- incrementReadLatchesReleased();
- bufferCache.unpin(node);
- incrementUnpins();
+ ctx.leafFrame.setPage(node);
+ succeed = true;
+ return node;
}
-
- pageId = ctx.pathList.getLastPageId();
- if (pageId != rootPage) {
- parentLsn = ctx.pathList.getPageLsn(ctx.pathList.size() - 2);
- }
- ctx.pathList.moveLast();
- continue;
}
-
- pageLsn = ctx.interiorFrame.getPageLsn();
- ctx.pathList.add(pageId, pageLsn, -1);
-
- if (!isLeaf) {
- // findBestChild must be called *before* getBestChildPageId
- ctx.interiorFrame.findBestChild(ctx.getTuple(), cmp);
- int childPageId = ctx.interiorFrame.getBestChildPageId();
-
- if (!writeLatched) {
+ } finally {
+ if (!succeed) {
+ if (readLatched) {
node.releaseReadLatch();
+ readLatched = false;
incrementReadLatchesReleased();
- // TODO: do we need to un-pin and pin again?
bufferCache.unpin(node);
incrementUnpins();
-
- node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
- incrementPins();
- node.acquireWriteLatch();
- incrementWriteLatchesAcquired();
- ctx.interiorFrame.setPage(node);
- writeLatched = true;
-
- if (ctx.interiorFrame.getPageLsn() != pageLsn) {
- // The page was changed while we unlocked it; thus,
- // retry (re-choose best child)
-
- ctx.pathList.moveLast();
- continue;
- }
+ } else if (writeLatched) {
+ node.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
}
-
- // We don't need to reset the frameTuple because it is
- // already pointing to the best child
- ctx.interiorFrame.enlarge(ctx.getTuple(), cmp);
-
- node.releaseWriteLatch();
- incrementWriteLatchesReleased();
- bufferCache.unpin(node);
- incrementUnpins();
- writeLatched = false;
-
- pageId = childPageId;
- parentLsn = pageLsn;
- } else {
- ctx.leafFrame.setPage(node);
- return node;
}
}
}
private void insertTuple(ICachedPage node, int pageId, ITupleReference tuple, RTreeOpContext ctx, boolean isLeaf)
- throws HyracksDataException, TreeIndexException {
- FrameOpSpaceStatus spaceStatus;
+ throws HyracksDataException, TreeIndexException, PageAllocationException {
+ FrameOpSpaceStatus spaceStatus;
if (!isLeaf) {
spaceStatus = ctx.interiorFrame.hasSpaceInsert(tuple);
} else {
@@ -525,54 +552,71 @@
}
}
- public void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ public void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ boolean writeLatched = false;
int parentId = ctx.pathList.getLastPageId();
ICachedPage parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
incrementPins();
parentNode.acquireWriteLatch();
+ writeLatched = true;
incrementWriteLatchesAcquired();
ctx.interiorFrame.setPage(parentNode);
boolean foundParent = true;
- if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
- foundParent = false;
- while (true) {
- if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp) != -1) {
- // found the parent
- foundParent = true;
- break;
+ try {
+
+ if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
+ foundParent = false;
+ while (true) {
+ if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp) != -1) {
+ // found the parent
+ foundParent = true;
+ break;
+ }
+ int rightPage = ctx.interiorFrame.getRightPage();
+ parentNode.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(parentNode);
+ incrementUnpins();
+
+ if (rightPage == -1) {
+ break;
+ }
+
+ parentId = rightPage;
+ parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
+ incrementPins();
+ parentNode.acquireWriteLatch();
+ writeLatched = true;
+ incrementWriteLatchesAcquired();
+ ctx.interiorFrame.setPage(parentNode);
}
- int rightPage = ctx.interiorFrame.getRightPage();
+ }
+ if (foundParent) {
+ ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), -1, cmp);
+ insertTuple(parentNode, parentId, ctx.splitKey.getRightTuple(), ctx, ctx.interiorFrame.isLeaf());
+ ctx.pathList.moveLast();
+
parentNode.releaseWriteLatch();
+ writeLatched = false;
incrementWriteLatchesReleased();
bufferCache.unpin(parentNode);
incrementUnpins();
+ return;
+ }
- if (rightPage == -1) {
- break;
- }
-
- parentId = rightPage;
- parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
- incrementPins();
- parentNode.acquireWriteLatch();
- incrementWriteLatchesAcquired();
- ctx.interiorFrame.setPage(parentNode);
+ } finally {
+ if (writeLatched) {
+ parentNode.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(parentNode);
+ incrementUnpins();
}
}
- if (foundParent) {
- ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), -1, cmp);
- insertTuple(parentNode, parentId, ctx.splitKey.getRightTuple(), ctx, ctx.interiorFrame.isLeaf());
- ctx.pathList.moveLast();
-
- parentNode.releaseWriteLatch();
- incrementWriteLatchesReleased();
- bufferCache.unpin(parentNode);
- incrementUnpins();
- return;
- }
-
- // very rare situation when the there is a root split, do an exhaustive
+ // very rare situation when the there is a root split, do an
+ // exhaustive
// breadth-first traversal looking for the parent tuple
ctx.pathList.clear();
@@ -582,48 +626,57 @@
}
public void findPath(RTreeOpContext ctx) throws HyracksDataException {
+ boolean readLatched = false;
int pageId = rootPage;
int parentIndex = -1;
long parentLsn = 0;
long pageLsn;
int pageIndex;
+ ICachedPage node = null;
ctx.traverseList.add(pageId, -1, parentIndex);
- while (!ctx.traverseList.isLast()) {
- pageId = ctx.traverseList.getFirstPageId();
- parentIndex = ctx.traverseList.getFirstPageIndex();
+ try {
+ while (!ctx.traverseList.isLast()) {
+ pageId = ctx.traverseList.getFirstPageId();
+ parentIndex = ctx.traverseList.getFirstPageIndex();
- ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
- incrementPins();
- node.acquireReadLatch();
- incrementReadLatchesAcquired();
- ctx.interiorFrame.setPage(node);
- pageLsn = ctx.interiorFrame.getPageLsn();
- pageIndex = ctx.traverseList.first();
- ctx.traverseList.setPageLsn(pageIndex, pageLsn);
+ node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ incrementPins();
+ node.acquireReadLatch();
+ readLatched = true;
+ incrementReadLatchesAcquired();
+ ctx.interiorFrame.setPage(node);
+ pageLsn = ctx.interiorFrame.getPageLsn();
+ pageIndex = ctx.traverseList.first();
+ ctx.traverseList.setPageLsn(pageIndex, pageLsn);
- ctx.traverseList.moveFirst();
+ ctx.traverseList.moveFirst();
- if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
- int rightPage = ctx.interiorFrame.getRightPage();
- if (rightPage != -1) {
- ctx.traverseList.add(rightPage, -1, parentIndex);
+ if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
+ int rightPage = ctx.interiorFrame.getRightPage();
+ if (rightPage != -1) {
+ ctx.traverseList.add(rightPage, -1, parentIndex);
+ }
}
- }
- parentLsn = pageLsn;
+ parentLsn = pageLsn;
- if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), ctx.traverseList, pageIndex, cmp) != -1) {
- fillPath(ctx, pageIndex);
-
+ if (ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), ctx.traverseList, pageIndex, cmp) != -1) {
+ fillPath(ctx, pageIndex);
+ return;
+ }
node.releaseReadLatch();
+ readLatched = false;
incrementReadLatchesReleased();
bufferCache.unpin(node);
incrementUnpins();
- return;
}
- node.releaseReadLatch();
- incrementReadLatchesReleased();
- bufferCache.unpin(node);
- incrementUnpins();
+ } finally {
+ if (readLatched) {
+ node.releaseReadLatch();
+ readLatched = false;
+ incrementReadLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
+ }
}
}
@@ -634,9 +687,7 @@
}
}
- @Override
- public void delete(ITupleReference tuple, IIndexOpContext ictx) throws HyracksDataException, TreeIndexException {
- RTreeOpContext ctx = (RTreeOpContext) ictx;
+ public void delete(ITupleReference tuple, RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
ctx.reset();
ctx.setTuple(tuple);
ctx.splitKey.reset();
@@ -665,70 +716,85 @@
}
public void updateParentForDelete(RTreeOpContext ctx) throws HyracksDataException {
+ boolean writeLatched = false;
int parentId = ctx.pathList.getLastPageId();
ICachedPage parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
incrementPins();
parentNode.acquireWriteLatch();
+ writeLatched = true;
incrementWriteLatchesAcquired();
ctx.interiorFrame.setPage(parentNode);
boolean foundParent = true;
int tupleIndex = -1;
- if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
- foundParent = false;
- while (true) {
- tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
- if (tupleIndex != -1) {
- // found the parent
- foundParent = true;
- break;
+ try {
+ if (ctx.interiorFrame.getPageLsn() != ctx.pathList.getLastPageLsn()) {
+ foundParent = false;
+ while (true) {
+ tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
+ if (tupleIndex != -1) {
+ // found the parent
+ foundParent = true;
+ break;
+ }
+ int rightPage = ctx.interiorFrame.getRightPage();
+ parentNode.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(parentNode);
+ incrementUnpins();
+
+ if (rightPage == -1) {
+ break;
+ }
+
+ parentId = rightPage;
+ parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
+ incrementPins();
+ parentNode.acquireWriteLatch();
+ writeLatched = true;
+ incrementWriteLatchesAcquired();
+ ctx.interiorFrame.setPage(parentNode);
}
- int rightPage = ctx.interiorFrame.getRightPage();
+ }
+ if (foundParent) {
+ if (tupleIndex == -1) {
+ tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
+ }
+ boolean recomputeMBR = ctx.interiorFrame.recomputeMBR(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
+
+ if (recomputeMBR) {
+ ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
+ ctx.pathList.moveLast();
+
+ incrementGlobalNsn();
+ ctx.interiorFrame.setPageLsn(getGlobalNsn());
+
+ ctx.splitKey.reset();
+ if (!ctx.pathList.isEmpty()) {
+ ctx.interiorFrame.computeMBR(ctx.splitKey);
+ ctx.splitKey.setLeftPage(parentId);
+ }
+ } else {
+ ctx.pathList.moveLast();
+ ctx.splitKey.reset();
+ }
+
parentNode.releaseWriteLatch();
+ writeLatched = false;
incrementWriteLatchesReleased();
bufferCache.unpin(parentNode);
incrementUnpins();
-
- if (rightPage == -1) {
- break;
- }
-
- parentId = rightPage;
- parentNode = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, parentId), false);
- incrementPins();
- parentNode.acquireWriteLatch();
- incrementWriteLatchesAcquired();
- ctx.interiorFrame.setPage(parentNode);
+ return;
}
- }
- if (foundParent) {
- if (tupleIndex == -1) {
- tupleIndex = ctx.interiorFrame.findTupleByPointer(ctx.splitKey.getLeftTuple(), cmp);
+ } finally {
+ if (writeLatched) {
+ parentNode.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(parentNode);
+ incrementUnpins();
}
- boolean recomputeMBR = ctx.interiorFrame.recomputeMBR(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
-
- if (recomputeMBR) {
- ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), tupleIndex, cmp);
- ctx.pathList.moveLast();
-
- incrementGlobalNsn();
- ctx.interiorFrame.setPageLsn(getGlobalNsn());
-
- ctx.splitKey.reset();
- if (!ctx.pathList.isEmpty()) {
- ctx.interiorFrame.computeMBR(ctx.splitKey);
- ctx.splitKey.setLeftPage(parentId);
- }
- } else {
- ctx.pathList.moveLast();
- ctx.splitKey.reset();
- }
-
- parentNode.releaseWriteLatch();
- incrementWriteLatchesReleased();
- bufferCache.unpin(parentNode);
- incrementUnpins();
- return;
}
// very rare situation when the there is a root split, do an exhaustive
@@ -741,87 +807,116 @@
}
public int findTupleToDelete(RTreeOpContext ctx) throws HyracksDataException {
-
+ boolean writeLatched = false;
+ boolean readLatched = false;
+ boolean succeed = false;
+ ICachedPage node = null;
ctx.traverseList.add(rootPage, -1, -1);
ctx.pathList.add(rootPage, -1, ctx.traverseList.size() - 1);
- while (!ctx.pathList.isEmpty()) {
- int pageId = ctx.pathList.getLastPageId();
- long parentLsn = ctx.pathList.getLastPageLsn();
- int pageIndex = ctx.pathList.getLastPageIndex();
- ctx.pathList.moveLast();
- ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
- incrementPins();
- node.acquireReadLatch();
- incrementReadLatchesAcquired();
- ctx.interiorFrame.setPage(node);
- boolean isLeaf = ctx.interiorFrame.isLeaf();
- long pageLsn = ctx.interiorFrame.getPageLsn();
- int parentIndex = ctx.traverseList.getPageIndex(pageIndex);
- ctx.traverseList.setPageLsn(pageIndex, pageLsn);
+ try {
+ while (!ctx.pathList.isEmpty()) {
+ int pageId = ctx.pathList.getLastPageId();
+ long parentLsn = ctx.pathList.getLastPageLsn();
+ int pageIndex = ctx.pathList.getLastPageIndex();
+ ctx.pathList.moveLast();
+ node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ incrementPins();
+ node.acquireReadLatch();
+ readLatched = true;
+ incrementReadLatchesAcquired();
+ ctx.interiorFrame.setPage(node);
+ boolean isLeaf = ctx.interiorFrame.isLeaf();
+ long pageLsn = ctx.interiorFrame.getPageLsn();
+ int parentIndex = ctx.traverseList.getPageIndex(pageIndex);
+ ctx.traverseList.setPageLsn(pageIndex, pageLsn);
- if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
- // Concurrent split detected, we need to visit the right page
- int rightPage = ctx.interiorFrame.getRightPage();
- if (rightPage != -1) {
- ctx.traverseList.add(rightPage, -1, parentIndex);
- ctx.pathList.add(rightPage, parentLsn, ctx.traverseList.size() - 1);
- }
- }
-
- if (!isLeaf) {
- for (int i = 0; i < ctx.interiorFrame.getTupleCount(); i++) {
- int childPageId = ctx.interiorFrame.getChildPageIdIfIntersect(ctx.tuple, i, cmp);
- if (childPageId != -1) {
- ctx.traverseList.add(childPageId, -1, pageIndex);
- ctx.pathList.add(childPageId, pageLsn, ctx.traverseList.size() - 1);
+ if (pageId != rootPage && parentLsn < ctx.interiorFrame.getPageNsn()) {
+ // Concurrent split detected, we need to visit the right
+ // page
+ int rightPage = ctx.interiorFrame.getRightPage();
+ if (rightPage != -1) {
+ ctx.traverseList.add(rightPage, -1, parentIndex);
+ ctx.pathList.add(rightPage, parentLsn, ctx.traverseList.size() - 1);
}
}
- } else {
- ctx.leafFrame.setPage(node);
- int tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
- if (tupleIndex != -1) {
- node.releaseReadLatch();
- incrementReadLatchesReleased();
- bufferCache.unpin(node);
- incrementUnpins();
-
- node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
- incrementPins();
- node.acquireWriteLatch();
- incrementWriteLatchesAcquired();
+ if (!isLeaf) {
+ for (int i = 0; i < ctx.interiorFrame.getTupleCount(); i++) {
+ int childPageId = ctx.interiorFrame.getChildPageIdIfIntersect(ctx.tuple, i, cmp);
+ if (childPageId != -1) {
+ ctx.traverseList.add(childPageId, -1, pageIndex);
+ ctx.pathList.add(childPageId, pageLsn, ctx.traverseList.size() - 1);
+ }
+ }
+ } else {
ctx.leafFrame.setPage(node);
+ int tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
+ if (tupleIndex != -1) {
- if (ctx.leafFrame.getPageLsn() != pageLsn) {
- // The page was changed while we unlocked it
+ node.releaseReadLatch();
+ readLatched = false;
+ incrementReadLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
- tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
- if (tupleIndex == -1) {
- ctx.traverseList.add(pageId, -1, parentIndex);
- ctx.pathList.add(pageId, parentLsn, ctx.traverseList.size() - 1);
+ node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ incrementPins();
+ node.acquireWriteLatch();
+ writeLatched = true;
+ incrementWriteLatchesAcquired();
+ ctx.leafFrame.setPage(node);
- node.releaseWriteLatch();
- incrementWriteLatchesReleased();
- bufferCache.unpin(node);
- incrementUnpins();
- continue;
+ if (ctx.leafFrame.getPageLsn() != pageLsn) {
+ // The page was changed while we unlocked it
+
+ tupleIndex = ctx.leafFrame.findTupleIndex(ctx.tuple, cmp);
+ if (tupleIndex == -1) {
+ ctx.traverseList.add(pageId, -1, parentIndex);
+ ctx.pathList.add(pageId, parentLsn, ctx.traverseList.size() - 1);
+
+ node.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
+ continue;
+ } else {
+ ctx.pathList.clear();
+ fillPath(ctx, pageIndex);
+ succeed = true;
+ return tupleIndex;
+ }
} else {
ctx.pathList.clear();
fillPath(ctx, pageIndex);
+ succeed = true;
return tupleIndex;
}
- } else {
- ctx.pathList.clear();
- fillPath(ctx, pageIndex);
- return tupleIndex;
}
}
+ node.releaseReadLatch();
+ readLatched = false;
+ incrementReadLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
}
- node.releaseReadLatch();
- incrementReadLatchesReleased();
- bufferCache.unpin(node);
- incrementUnpins();
+ } finally {
+ if (!succeed) {
+ if (readLatched) {
+ node.releaseReadLatch();
+ readLatched = false;
+ incrementReadLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
+ } else if (writeLatched) {
+ node.releaseWriteLatch();
+ writeLatched = false;
+ incrementWriteLatchesReleased();
+ bufferCache.unpin(node);
+ incrementUnpins();
+ }
+ }
}
return -1;
}
@@ -838,14 +933,14 @@
}
}
- public void search(ITreeIndexCursor cursor, SearchPredicate pred, RTreeOpContext ctx) throws Exception {
- ctx.reset();
+ private void search(ITreeIndexCursor cursor, ISearchPredicate searchPred, RTreeOpContext ctx) throws HyracksDataException, TreeIndexException {
+ ctx.reset();
ctx.cursor = cursor;
cursor.setBufferCache(bufferCache);
cursor.setFileId(fileId);
ctx.cursorInitialState.setRootPage(rootPage);
- ctx.cursor.open(ctx.cursorInitialState, pred);
+ ctx.cursor.open(ctx.cursorInitialState, (SearchPredicate)searchPred);
}
public ITreeIndexFrameFactory getInteriorFrameFactory() {
@@ -864,18 +959,17 @@
return freePageManager;
}
- @Override
- public void update(ITupleReference tuple, IIndexOpContext ictx) {
+ private void update(ITupleReference tuple, RTreeOpContext ctx) {
throw new UnsupportedOperationException("RTree Update not implemented.");
}
public final class BulkLoadContext implements IIndexBulkLoadContext {
- public RTreeOpContext insertOpCtx;
+ public ITreeIndexAccessor indexAccessor;
public BulkLoadContext(float fillFactor, IRTreeFrame leafFrame, IRTreeFrame interiorFrame,
ITreeIndexMetaDataFrame metaFrame) throws HyracksDataException {
- insertOpCtx = createOpContext(IndexOp.INSERT);
+ indexAccessor = createAccessor();
}
}
@@ -885,15 +979,16 @@
throw new HyracksDataException("Trying to bulk-load RTree but RTree has already been loaded.");
}
- BulkLoadContext ctx = new BulkLoadContext(fillFactor, (IRTreeFrame) leafFrameFactory.createFrame(), (IRTreeFrame) interiorFrameFactory.createFrame(),
- freePageManager.getMetaDataFrameFactory().createFrame());
+ BulkLoadContext ctx = new BulkLoadContext(fillFactor, (IRTreeFrame) leafFrameFactory.createFrame(),
+ (IRTreeFrame) interiorFrameFactory.createFrame(), freePageManager.getMetaDataFrameFactory()
+ .createFrame());
return ctx;
}
@Override
public void bulkLoadAddTuple(ITupleReference tuple, IIndexBulkLoadContext ictx) throws HyracksDataException {
try {
- insert(tuple, ((BulkLoadContext) ictx).insertOpCtx);
+ ((BulkLoadContext) ictx).indexAccessor.insert(tuple);
} catch (Exception e) {
throw new HyracksDataException("BulkLoad Error");
}
@@ -904,10 +999,8 @@
loaded = true;
}
- @Override
- public void diskOrderScan(ITreeIndexCursor icursor, IIndexOpContext ictx) throws HyracksDataException {
+ private void diskOrderScan(ITreeIndexCursor icursor, RTreeOpContext ctx) throws HyracksDataException {
TreeDiskOrderScanCursor cursor = (TreeDiskOrderScanCursor) icursor;
- RTreeOpContext ctx = (RTreeOpContext) ictx;
ctx.reset();
int currentPageId = rootPage + 1;
@@ -915,12 +1008,18 @@
ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
page.acquireReadLatch();
- cursor.setBufferCache(bufferCache);
- cursor.setFileId(fileId);
- cursor.setCurrentPageId(currentPageId);
- cursor.setMaxPageId(maxPageId);
- ctx.cursorInitialState.setPage(page);
- cursor.open(ctx.cursorInitialState, diskOrderScanPredicate);
+ try {
+ cursor.setBufferCache(bufferCache);
+ cursor.setFileId(fileId);
+ cursor.setCurrentPageId(currentPageId);
+ cursor.setMaxPageId(maxPageId);
+ ctx.cursorInitialState.setPage(page);
+ cursor.open(ctx.cursorInitialState, diskOrderScanPredicate);
+ } catch (Exception e) {
+ page.releaseReadLatch();
+ bufferCache.unpin(page);
+ throw new HyracksDataException(e);
+ }
}
@Override
@@ -937,4 +1036,50 @@
public IndexType getIndexType() {
return IndexType.RTREE;
}
-}
+
+ @Override
+ public ITreeIndexAccessor createAccessor() {
+ return new RTreeAccessor(this);
+ }
+
+ private class RTreeAccessor implements ITreeIndexAccessor {
+ private RTree rtree;
+ private RTreeOpContext ctx;
+
+ public RTreeAccessor(RTree rtree) {
+ this.rtree = rtree;
+ this.ctx = rtree.createOpContext();
+ }
+
+ @Override
+ public void insert(ITupleReference tuple) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ ctx.reset(IndexOp.INSERT);
+ rtree.insert(tuple, ctx);
+ }
+
+ @Override
+ public void update(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.UPDATE);
+ rtree.update(tuple, ctx);
+ }
+
+ @Override
+ public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndexException {
+ ctx.reset(IndexOp.DELETE);
+ rtree.delete(tuple, ctx);
+ }
+
+ @Override
+ public void search(ITreeIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException,
+ TreeIndexException {
+ ctx.reset(IndexOp.SEARCH);
+ rtree.search(cursor, searchPred, ctx);
+ }
+
+ @Override
+ public void diskOrderScan(ITreeIndexCursor cursor) throws HyracksDataException {
+ ctx.reset(IndexOp.DISKORDERSCAN);
+ rtree.diskOrderScan(cursor, ctx);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
index dc4a753..c258377 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeOpContext.java
@@ -37,24 +37,13 @@
public PathList traverseList; // used for traversing the tree
private static final int initTraverseListSize = 100;
- public RTreeOpContext(IndexOp op, IRTreeLeafFrame leafFrame,
+ public RTreeOpContext(IRTreeLeafFrame leafFrame,
IRTreeInteriorFrame interiorFrame,
ITreeIndexMetaDataFrame metaFrame, int treeHeightHint) {
this.interiorFrame = interiorFrame;
this.leafFrame = leafFrame;
this.metaFrame = metaFrame;
pathList = new PathList(treeHeightHint, treeHeightHint);
- if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
- splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter()
- .createTupleReference(), interiorFrame.getTupleWriter()
- .createTupleReference());
- traverseList = new PathList(initTraverseListSize,
- initTraverseListSize);
- } else {
- splitKey = null;
- traverseList = null;
- cursorInitialState = new RTreeCursorInitialState(pathList, 1);
- }
}
public ITupleReference getTuple() {
@@ -76,6 +65,9 @@
@Override
public void reset(IndexOp newOp) {
+ if (op != null && newOp == op) {
+ return;
+ }
if (op != IndexOp.SEARCH && op != IndexOp.DISKORDERSCAN) {
if (splitKey == null) {
splitKey = new RTreeSplitKey(interiorFrame.getTupleWriter()
@@ -86,11 +78,9 @@
traverseList = new PathList(initTraverseListSize,
initTraverseListSize);
}
-
- } else {
- if (cursorInitialState == null) {
- cursorInitialState = new RTreeCursorInitialState(pathList, 1);
- }
+ }
+ if (cursorInitialState == null) {
+ cursorInitialState = new RTreeCursorInitialState(pathList, 1);
}
this.op = newOp;
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index a138212..e65f1d4 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -30,188 +30,198 @@
public class RTreeSearchCursor implements ITreeIndexCursor {
- private int fileId = -1;
- private ICachedPage page = null;
- private IRTreeInteriorFrame interiorFrame = null;
- private IRTreeLeafFrame leafFrame = null;
- private IBufferCache bufferCache = null;
+ private int fileId = -1;
+ private ICachedPage page = null;
+ private IRTreeInteriorFrame interiorFrame = null;
+ private IRTreeLeafFrame leafFrame = null;
+ private IBufferCache bufferCache = null;
- private SearchPredicate pred;
- private PathList pathList;
- private int rootPage;
- ITupleReference searchKey;
+ private SearchPredicate pred;
+ private PathList pathList;
+ private int rootPage;
+ ITupleReference searchKey;
- private int tupleIndex = 0;
- private int tupleIndexInc = 0;
+ private int tupleIndex = 0;
+ private int tupleIndexInc = 0;
- private MultiComparator cmp;
+ private MultiComparator cmp;
- private ITreeIndexTupleReference frameTuple;
- private boolean readLatched = false;
+ private ITreeIndexTupleReference frameTuple;
+ private boolean readLatched = false;
- private int pin = 0;
- private int unpin = 0;
+ private int pin = 0;
+ private int unpin = 0;
- public RTreeSearchCursor(IRTreeInteriorFrame interiorFrame,
- IRTreeLeafFrame leafFrame) {
- this.interiorFrame = interiorFrame;
- this.leafFrame = leafFrame;
- this.frameTuple = leafFrame.createTupleReference();
- }
+ public RTreeSearchCursor(IRTreeInteriorFrame interiorFrame, IRTreeLeafFrame leafFrame) {
+ this.interiorFrame = interiorFrame;
+ this.leafFrame = leafFrame;
+ this.frameTuple = leafFrame.createTupleReference();
+ }
+
+ @Override
+ public void close() throws Exception {
+ if (readLatched) {
+ page.releaseReadLatch();
+ bufferCache.unpin(page);
+ readLatched = false;
+ }
+ tupleIndex = 0;
+ tupleIndexInc = 0;
+ page = null;
+ pathList = null;
+ }
+
+ public ITupleReference getTuple() {
+ return frameTuple;
+ }
+
+ @Override
+ public ICachedPage getPage() {
+ return page;
+ }
+
+ public boolean fetchNextLeafPage() throws HyracksDataException {
+ boolean succeed = false;
+ if (readLatched) {
+ page.releaseReadLatch();
+ bufferCache.unpin(page);
+ unpin++;
+ readLatched = false;
+ }
+
+ while (!pathList.isEmpty()) {
+ int pageId = pathList.getLastPageId();
+ long parentLsn = pathList.getLastPageLsn();
+ pathList.moveLast();
+ ICachedPage node = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ pin++;
+ node.acquireReadLatch();
+ readLatched = true;
+ try {
+ interiorFrame.setPage(node);
+ boolean isLeaf = interiorFrame.isLeaf();
+ long pageLsn = interiorFrame.getPageLsn();
+
+ if (pageId != rootPage && parentLsn < interiorFrame.getPageNsn()) {
+ // Concurrent split detected, we need to visit the right
+ // page
+ int rightPage = interiorFrame.getRightPage();
+ if (rightPage != -1) {
+ pathList.add(rightPage, parentLsn, -1);
+ }
+ }
+
+ if (!isLeaf) {
+ for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
+ int childPageId = interiorFrame.getChildPageIdIfIntersect(searchKey, i, cmp);
+ if (childPageId != -1) {
+ pathList.add(childPageId, pageLsn, -1);
+ }
+ }
+ } else {
+ page = node;
+ leafFrame.setPage(page);
+ tupleIndex = 0;
+ succeed = true;
+ return true;
+ }
+ } finally {
+ if (!succeed) {
+ if (readLatched) {
+ node.releaseReadLatch();
+ readLatched = false;
+ bufferCache.unpin(node);
+ unpin++;
+ }
+ }
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public boolean hasNext() throws Exception {
+ if (page == null) {
+ return false;
+ }
+
+ if (tupleIndex == leafFrame.getTupleCount()) {
+ if (!fetchNextLeafPage()) {
+ return false;
+ }
+ }
+
+ do {
+ for (int i = tupleIndex; i < leafFrame.getTupleCount(); i++) {
+ if (leafFrame.intersect(searchKey, i, cmp)) {
+ frameTuple.resetByTupleIndex(leafFrame, i);
+ tupleIndexInc = i + 1;
+ return true;
+ }
+ }
+ } while (fetchNextLeafPage());
+ return false;
+ }
+
+ @Override
+ public void next() throws Exception {
+ tupleIndex = tupleIndexInc;
+ }
+
+ @Override
+ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+ // in case open is called multiple times without closing
+ if (this.page != null) {
+ this.page.releaseReadLatch();
+ readLatched = false;
+ bufferCache.unpin(this.page);
+ pathList.clear();
+ }
+
+ pathList = ((RTreeCursorInitialState) initialState).getPathList();
+ rootPage = ((RTreeCursorInitialState) initialState).getRootPage();
+
+ pred = (SearchPredicate) searchPred;
+ cmp = pred.getLowKeyComparator();
+ searchKey = pred.getSearchKey();
+
+ int maxFieldPos = cmp.getKeyFieldCount() / 2;
+ for (int i = 0; i < maxFieldPos; i++) {
+ int j = maxFieldPos + i;
+ int c = cmp.getComparators()[i].compare(searchKey.getFieldData(i), searchKey.getFieldStart(i),
+ searchKey.getFieldLength(i), searchKey.getFieldData(j), searchKey.getFieldStart(j),
+ searchKey.getFieldLength(j));
+ if (c > 0) {
+ throw new IllegalArgumentException("The low key point has larger coordinates than the high key point.");
+ }
+ }
+
+ pathList.add(this.rootPage, -1, -1);
+ tupleIndex = 0;
+ fetchNextLeafPage();
+ }
+
+ @Override
+ public void reset() {
+ try {
+ close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void setBufferCache(IBufferCache bufferCache) {
+ this.bufferCache = bufferCache;
+ }
+
+ @Override
+ public void setFileId(int fileId) {
+ this.fileId = fileId;
+ }
@Override
- public void close() throws Exception {
- if (readLatched) {
- page.releaseReadLatch();
- bufferCache.unpin(page);
- readLatched = false;
- }
- tupleIndex = 0;
- tupleIndexInc = 0;
- page = null;
- pathList = null;
- }
-
- public ITupleReference getTuple() {
- return frameTuple;
- }
-
- @Override
- public ICachedPage getPage() {
- return page;
- }
-
- public boolean fetchNextLeafPage() throws HyracksDataException {
- if (readLatched) {
- page.releaseReadLatch();
- bufferCache.unpin(page);
- unpin++;
- readLatched = false;
- }
- while (!pathList.isEmpty()) {
- int pageId = pathList.getLastPageId();
- long parentLsn = pathList.getLastPageLsn();
- pathList.moveLast();
- ICachedPage node = bufferCache.pin(
- BufferedFileHandle.getDiskPageId(fileId, pageId), false);
- pin++;
- node.acquireReadLatch();
- readLatched = true;
- interiorFrame.setPage(node);
- boolean isLeaf = interiorFrame.isLeaf();
- long pageLsn = interiorFrame.getPageLsn();
-
- if (pageId != rootPage && parentLsn < interiorFrame.getPageNsn()) {
- // Concurrent split detected, we need to visit the right page
- int rightPage = interiorFrame.getRightPage();
- if (rightPage != -1) {
- pathList.add(rightPage, parentLsn, -1);
- }
- }
-
- if (!isLeaf) {
- for (int i = 0; i < interiorFrame.getTupleCount(); i++) {
- int childPageId = interiorFrame.getChildPageIdIfIntersect(
- searchKey, i, cmp);
- if (childPageId != -1) {
- pathList.add(childPageId, pageLsn, -1);
- }
- }
- } else {
- page = node;
- leafFrame.setPage(page);
- tupleIndex = 0;
- return true;
- }
- node.releaseReadLatch();
- readLatched = false;
- bufferCache.unpin(node);
- unpin++;
- }
+ public boolean exclusiveLatchNodes() {
return false;
}
-
- @Override
- public boolean hasNext() throws Exception {
- if (page == null) {
- return false;
- }
-
- if (tupleIndex == leafFrame.getTupleCount()) {
- if (!fetchNextLeafPage()) {
- return false;
- }
- }
-
- do {
- for (int i = tupleIndex; i < leafFrame.getTupleCount(); i++) {
- if (leafFrame.intersect(searchKey, i, cmp)) {
- frameTuple.resetByTupleIndex(leafFrame, i);
- tupleIndexInc = i + 1;
- return true;
- }
- }
- } while (fetchNextLeafPage());
- return false;
- }
-
- @Override
- public void next() throws Exception {
- tupleIndex = tupleIndexInc;
- }
-
- @Override
- public void open(ICursorInitialState initialState,
- ISearchPredicate searchPred) throws Exception {
- // in case open is called multiple times without closing
- if (this.page != null) {
- this.page.releaseReadLatch();
- readLatched = false;
- bufferCache.unpin(this.page);
- pathList.clear();
- }
-
- pathList = ((RTreeCursorInitialState) initialState).getPathList();
- rootPage = ((RTreeCursorInitialState) initialState).getRootPage();
-
- pred = (SearchPredicate) searchPred;
- cmp = pred.getLowKeyComparator();
- searchKey = pred.getSearchKey();
-
- int maxFieldPos = cmp.getKeyFieldCount() / 2;
- for (int i = 0; i < maxFieldPos; i++) {
- int j = maxFieldPos + i;
- int c = cmp.getComparators()[i].compare(searchKey.getFieldData(i),
- searchKey.getFieldStart(i), searchKey.getFieldLength(i),
- searchKey.getFieldData(j), searchKey.getFieldStart(j),
- searchKey.getFieldLength(j));
- if (c > 0) {
- throw new IllegalArgumentException(
- "The low key point has larger coordinates than the high key point.");
- }
- }
-
- pathList.add(this.rootPage, -1, -1);
- tupleIndex = 0;
- fetchNextLeafPage();
- }
-
- @Override
- public void reset() {
- try {
- close();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public void setBufferCache(IBufferCache bufferCache) {
- this.bufferCache = bufferCache;
- }
-
- @Override
- public void setFileId(int fileId) {
- this.fileId = fileId;
- }
}
\ No newline at end of file
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 d7558ba..883b25e 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
@@ -132,6 +132,9 @@
pinSanityCheck(dpid);
CachedPage cPage = findPage(dpid, newPage);
+ if (cPage == null) {
+ throw new HyracksDataException("Failed to pin page because all pages are pinned.");
+ }
if (!newPage) {
if (!cPage.valid) {
/*
@@ -564,8 +567,11 @@
synchronized (fileInfoMap) {
try {
for(Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
- sweepAndFlush(entry.getKey());
- ioManager.close(entry.getValue().getFileHandle());
+ boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
+ sweepAndFlush(entry.getKey(), !fileHasBeenDeleted);
+ if (!fileHasBeenDeleted) {
+ ioManager.close(entry.getValue().getFileHandle());
+ }
}
} catch(HyracksDataException e) {
e.printStackTrace();
@@ -601,9 +607,12 @@
for(Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
if(entry.getValue().getReferenceCount() <= 0) {
int entryFileId = entry.getKey();
- sweepAndFlush(entryFileId);
+ boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
+ sweepAndFlush(entryFileId, !fileHasBeenDeleted);
+ if (!fileHasBeenDeleted) {
+ ioManager.close(entry.getValue().getFileHandle());
+ }
fileInfoMap.remove(entryFileId);
- ioManager.close(entry.getValue().getFileHandle());
unreferencedFileFound = true;
// for-each iterator is invalid because we changed fileInfoMap
break;
@@ -626,7 +635,7 @@
}
}
- private void sweepAndFlush(int fileId) throws HyracksDataException {
+ private void sweepAndFlush(int fileId, boolean flushDirtyPages) throws HyracksDataException {
for (int i = 0; i < pageMap.length; ++i) {
CacheBucket bucket = pageMap[i];
bucket.bucketLock.lock();
@@ -637,7 +646,7 @@
if (cPage == null) {
break;
}
- if (invalidateIfFileIdMatch(fileId, cPage)) {
+ if (invalidateIfFileIdMatch(fileId, cPage, flushDirtyPages)) {
prev.next = cPage.next;
cPage.next = null;
} else {
@@ -646,7 +655,7 @@
}
// Take care of the head of the chain.
if (bucket.cachedPage != null) {
- if (invalidateIfFileIdMatch(fileId, bucket.cachedPage)) {
+ if (invalidateIfFileIdMatch(fileId, bucket.cachedPage, flushDirtyPages)) {
CachedPage cPage = bucket.cachedPage;
bucket.cachedPage = bucket.cachedPage.next;
cPage.next = null;
@@ -658,14 +667,19 @@
}
}
- private boolean invalidateIfFileIdMatch(int fileId, CachedPage cPage) throws HyracksDataException {
+ private boolean invalidateIfFileIdMatch(int fileId, CachedPage cPage, boolean flushDirtyPages) throws HyracksDataException {
if (BufferedFileHandle.getFileId(cPage.dpid) == fileId) {
- if (cPage.dirty.get()) {
- write(cPage);
+ int pinCount;
+ if (cPage.dirty.get()) {
+ if (flushDirtyPages) {
+ write(cPage);
+ }
cPage.dirty.set(false);
- cPage.pinCount.decrementAndGet();
+ pinCount = cPage.pinCount.decrementAndGet();
+ } else {
+ pinCount = cPage.pinCount.get();
}
- if (cPage.pinCount.get() != 0) {
+ if (pinCount != 0) {
throw new IllegalStateException("Page is pinned and file is being closed");
}
cPage.invalidate();
@@ -697,11 +711,22 @@
LOGGER.info("Deleting file: " + fileId + " in cache: " + this);
}
synchronized (fileInfoMap) {
- BufferedFileHandle fInfo = fileInfoMap.get(fileId);
- if (fInfo != null) {
- throw new HyracksDataException("Deleting open file");
- }
- fileMapManager.unregisterFile(fileId);
+ BufferedFileHandle fInfo = null;
+ try {
+ fInfo = fileInfoMap.get(fileId);
+ if (fInfo != null && fInfo.getReferenceCount() > 0) {
+ throw new HyracksDataException("Deleting open file");
+ }
+ } finally {
+ fileMapManager.unregisterFile(fileId);
+ if (fInfo != null) {
+ // Mark the fInfo as deleted,
+ // such that when its pages are reclaimed in openFile(),
+ // the pages are not flushed to disk but only invalidates.
+ ioManager.close(fInfo.getFileHandle());
+ fInfo.markAsDeleted();
+ }
+ }
}
}
}
\ No newline at end of file
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
index 3137f20..ac062d2 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/file/BufferedFileHandle.java
@@ -37,6 +37,14 @@
return handle;
}
+ public void markAsDeleted() {
+ handle = null;
+ }
+
+ public boolean fileHasBeenDeleted() {
+ return handle == null;
+ }
+
public int incReferenceCount() {
return refCount.incrementAndGet();
}
diff --git a/hyracks-test-support/pom.xml b/hyracks-test-support/pom.xml
index 95b3937..fdcf7d4 100644
--- a/hyracks-test-support/pom.xml
+++ b/hyracks-test-support/pom.xml
@@ -43,5 +43,12 @@
<type>jar</type>
<scope>compile</scope>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-invertedindex</artifactId>
+ <version>0.2.0-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
similarity index 76%
rename from hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
rename to hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
index 9925fe8..27d50f5 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestIndexRegistryProvider.java
@@ -15,15 +15,15 @@
package edu.uci.ics.hyracks.test.support;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-public class TestTreeIndexRegistryProvider implements IIndexRegistryProvider<ITreeIndex> {
+public class TestIndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
private static final long serialVersionUID = 1L;
@Override
- public IndexRegistry<ITreeIndex> getRegistry(IHyracksTaskContext ctx) {
- return TestStorageManagerComponentHolder.getTreeIndexRegistry(ctx);
+ public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
+ return TestStorageManagerComponentHolder.getIndexRegistry(ctx);
}
}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index c7bd09d..d006f42 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.test.support;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -30,7 +30,7 @@
public class TestStorageManagerComponentHolder {
private static IBufferCache bufferCache;
private static IFileMapProvider fileMapProvider;
- private static IndexRegistry<ITreeIndex> treeIndexRegistry;
+ private static IndexRegistry<IIndex> indexRegistry;
private static int pageSize;
private static int numPages;
@@ -42,7 +42,7 @@
TestStorageManagerComponentHolder.maxOpenFiles = maxOpenFiles;
bufferCache = null;
fileMapProvider = null;
- treeIndexRegistry = null;
+ indexRegistry = null;
}
public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
@@ -63,10 +63,10 @@
return fileMapProvider;
}
- public synchronized static IndexRegistry<ITreeIndex> getTreeIndexRegistry(IHyracksTaskContext ctx) {
- if (treeIndexRegistry == null) {
- treeIndexRegistry = new IndexRegistry<ITreeIndex>();
+ public synchronized static IndexRegistry<IIndex> getIndexRegistry(IHyracksTaskContext ctx) {
+ if (indexRegistry == null) {
+ indexRegistry = new IndexRegistry<IIndex>();
}
- return treeIndexRegistry;
+ return indexRegistry;
}
}
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
new file mode 100644
index 0000000..c14a11d
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeExamplesTest.java
@@ -0,0 +1,623 @@
+/*
+ * 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;
+
+import java.util.Random;
+import java.util.logging.Level;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+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.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
+import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+
+@SuppressWarnings("rawtypes")
+public class BTreeExamplesTest extends AbstractBTreeTest {
+
+ /**
+ * Fixed-Length Key,Value Example.
+ *
+ * Create a BTree with one fixed-length key field and one fixed-length value
+ * field. Fill BTree with random values using insertions (not bulk load).
+ * Perform scans and range search.
+ */
+ @Test
+ public void fixedLengthKeyValueExample() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Fixed-Length Key,Value Example.");
+ }
+
+ // Declare fields.
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ // Declare field serdes.
+ ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+
+ // Declare keys.
+ int keyFieldCount = 1;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ BTree btree = BTreeUtils
+ .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ long start = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inserting into tree...");
+ }
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ int numInserts = 10000;
+ for (int i = 0; i < numInserts; i++) {
+ int f0 = rnd.nextInt() % numInserts;
+ int f1 = 5;
+ TupleUtils.createIntegerTuple(tb, tuple, f0, f1);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("Inserting " + i + " : " + f0 + " " + f1);
+ }
+ }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ }
+ }
+ long end = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(numInserts + " inserts in " + (end - start) + "ms");
+ }
+
+ orderedScan(btree, indexAccessor, fieldSerdes);
+ diskOrderScan(btree, indexAccessor, fieldSerdes);
+
+ // Build low key.
+ ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(keyFieldCount);
+ ArrayTupleReference lowKey = new ArrayTupleReference();
+ TupleUtils.createIntegerTuple(lowKeyTb, lowKey, -1000);
+
+ // Build high key.
+ ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(keyFieldCount);
+ ArrayTupleReference highKey = new ArrayTupleReference();
+ TupleUtils.createIntegerTuple(highKeyTb, highKey, 1000);
+
+ rangeSearch(btree, indexAccessor, fieldSerdes, lowKey, highKey);
+
+ btree.close();
+ }
+
+ /**
+ * Composite Key Example (Non-Unique B-Tree).
+ *
+ * Create a BTree with two fixed-length key fields and one fixed-length
+ * value field. Fill BTree with random values using insertions (not bulk
+ * load) Perform scans and range search.
+ */
+ @Test
+ public void twoFixedLengthKeysOneFixedLengthValueExample() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Composite Key Test");
+ }
+
+ // Declare fields.
+ int fieldCount = 3;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ typeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ // Declare field serdes.
+ ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+
+ // declare keys
+ int keyFieldCount = 2;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ BTree btree = BTreeUtils
+ .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ long start = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inserting into tree...");
+ }
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ int numInserts = 10000;
+ for (int i = 0; i < 10000; i++) {
+ int f0 = rnd.nextInt() % 2000;
+ int f1 = rnd.nextInt() % 1000;
+ int f2 = 5;
+ TupleUtils.createIntegerTuple(tb, tuple, f0, f1, f2);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("Inserting " + i + " : " + f0 + " " + f1 + " " + f2);
+ }
+ }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ }
+ }
+ long end = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(numInserts + " inserts in " + (end - start) + "ms");
+ }
+
+ orderedScan(btree, indexAccessor, fieldSerdes);
+ diskOrderScan(btree, indexAccessor, fieldSerdes);
+
+ // Build low key.
+ ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(1);
+ ArrayTupleReference lowKey = new ArrayTupleReference();
+ TupleUtils.createIntegerTuple(lowKeyTb, lowKey, -3);
+
+ // Build high key.
+ ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(1);
+ ArrayTupleReference highKey = new ArrayTupleReference();
+ TupleUtils.createIntegerTuple(highKeyTb, highKey, 3);
+
+ // Prefix-Range search in [-3, 3]
+ rangeSearch(btree, indexAccessor, fieldSerdes, lowKey, highKey);
+
+ btree.close();
+ }
+
+ /**
+ * Variable-Length Example. Create a BTree with one variable-length key
+ * field and one variable-length value field. Fill BTree with random values
+ * using insertions (not bulk load) Perform ordered scans and range search.
+ */
+ @Test
+ public void varLenKeyValueExample() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Variable-Length Key,Value Example");
+ }
+
+ // Declare fields.
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ // Declare field serdes.
+ ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+
+ // Declare keys.
+ int keyFieldCount = 1;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ BTree btree = BTreeUtils
+ .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ long start = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inserting into tree...");
+ }
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ // Max string length to be generated.
+ int maxLength = 10;
+ int numInserts = 10000;
+ for (int i = 0; i < 10000; i++) {
+ String f0 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ TupleUtils.createTuple(tb, tuple, fieldSerdes, f0, f1);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("Inserting " + f0 + " " + f1);
+ }
+ }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ }
+ }
+ long end = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(numInserts + " inserts in " + (end - start) + "ms");
+ }
+
+ orderedScan(btree, indexAccessor, fieldSerdes);
+ diskOrderScan(btree, indexAccessor, fieldSerdes);
+
+ // Build low key.
+ ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(1);
+ ArrayTupleReference lowKey = new ArrayTupleReference();
+ TupleUtils.createTuple(lowKeyTb, lowKey, fieldSerdes, "cbf");
+
+ // Build high key.
+ ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(1);
+ ArrayTupleReference highKey = new ArrayTupleReference();
+ TupleUtils.createTuple(highKeyTb, highKey, fieldSerdes, "cc7");
+
+ rangeSearch(btree, indexAccessor, fieldSerdes, lowKey, highKey);
+
+ btree.close();
+ }
+
+ /**
+ * Deletion Example.
+ *
+ * Create a BTree with one variable-length key field and one variable-length
+ * value field. Fill B-tree with random values using insertions, then delete
+ * entries one-by-one. Repeat procedure a few times on same BTree.
+ */
+ @Test
+ public void deleteExample() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Deletion Example");
+ }
+
+ // Declare fields.
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ // Declare field serdes.
+ ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+
+ // Declare keys.
+ int keyFieldCount = 1;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ BTree btree = BTreeUtils
+ .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ // Max string length to be generated.
+ int runs = 3;
+ for (int run = 0; run < runs; run++) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Deletion example run: " + (run + 1) + "/" + runs);
+ LOGGER.info("Inserting into tree...");
+ }
+ int maxLength = 10;
+ int ins = 10000;
+ String[] f0s = new String[ins];
+ String[] f1s = new String[ins];
+ int insDone = 0;
+ int[] insDoneCmp = new int[ins];
+ for (int i = 0; i < ins; i++) {
+ String f0 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ TupleUtils.createTuple(tb, tuple, fieldSerdes, f0, f1);
+ f0s[i] = f0;
+ f1s[i] = f1;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("Inserting " + i);
+ }
+ }
+ try {
+ indexAccessor.insert(tuple);
+ insDone++;
+ } catch (TreeIndexException e) {
+ }
+ insDoneCmp[i] = insDone;
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Deleting from tree...");
+ }
+ int delDone = 0;
+ for (int i = 0; i < ins; i++) {
+ TupleUtils.createTuple(tb, tuple, fieldSerdes, f0s[i], f1s[i]);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("Deleting " + i);
+ }
+ }
+ try {
+ indexAccessor.delete(tuple);
+ delDone++;
+ } catch (TreeIndexException e) {
+ }
+ if (insDoneCmp[i] != delDone) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("INCONSISTENT STATE, ERROR IN DELETION EXAMPLE.");
+ LOGGER.info("INSDONECMP: " + insDoneCmp[i] + " " + delDone);
+ }
+ break;
+ }
+ }
+ if (insDone != delDone) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("ERROR! INSDONE: " + insDone + " DELDONE: " + delDone);
+ }
+ break;
+ }
+ }
+ btree.close();
+ }
+
+ /**
+ * Update example.
+ *
+ * Create a BTree with one variable-length key field and one variable-length
+ * value field. Fill B-tree with random values using insertions, then update
+ * entries one-by-one. Repeat procedure a few times on same BTree.
+ */
+ @Test
+ public void updateExample() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Update example");
+ }
+
+ // Declare fields.
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ typeTraits[1] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ // Declare field serdes.
+ ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
+
+ // Declare keys.
+ int keyFieldCount = 1;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ BTree btree = BTreeUtils
+ .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inserting into tree...");
+ }
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ int maxLength = 10;
+ int ins = 10000;
+ String[] keys = new String[10000];
+ for (int i = 0; i < ins; i++) {
+ String f0 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ TupleUtils.createTuple(tb, tuple, fieldSerdes, f0, f1);
+ keys[i] = f0;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("Inserting " + i);
+ }
+ }
+ try {
+ indexAccessor.insert(tuple);
+ } catch (TreeIndexException e) {
+ }
+ }
+ // Print before doing any updates.
+ orderedScan(btree, indexAccessor, fieldSerdes);
+
+ int runs = 3;
+ for (int run = 0; run < runs; run++) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Update test run: " + (run + 1) + "/" + runs);
+ LOGGER.info("Updating BTree");
+ }
+ for (int i = 0; i < ins; i++) {
+ // Generate a new random value for f1.
+ String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
+ TupleUtils.createTuple(tb, tuple, fieldSerdes, keys[i], f1);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("UPDATING " + i);
+ }
+ }
+ try {
+ indexAccessor.update(tuple);
+ } catch (TreeIndexException e) {
+ }
+ }
+ // Do another scan after a round of updates.
+ orderedScan(btree, indexAccessor, fieldSerdes);
+ }
+ btree.close();
+ }
+
+ /**
+ * Bulk load example.
+ *
+ * Load a tree with 100,000 tuples. BTree has a composite key to "simulate"
+ * non-unique index creation.
+ *
+ */
+ @Test
+ public void bulkLoadExample() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Bulk load example");
+ }
+ // Declare fields.
+ int fieldCount = 3;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ typeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ typeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ // Declare field serdes.
+ ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+
+ // declare keys
+ int keyFieldCount = 2;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+ cmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+ BTree btree = BTreeUtils
+ .createBTree(bufferCache, btreeFileId, typeTraits, cmps, BTreeLeafFrameType.REGULAR_NSM);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ // Load sorted records.
+ int ins = 100000;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Bulk loading " + ins + " tuples");
+ }
+ long start = System.currentTimeMillis();
+ IIndexBulkLoadContext bulkLoadCtx = btree.beginBulkLoad(0.7f);
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference tuple = new ArrayTupleReference();
+ for (int i = 0; i < ins; i++) {
+ TupleUtils.createIntegerTuple(tb, tuple, i, i, 5);
+ btree.bulkLoadAddTuple(tuple, bulkLoadCtx);
+ }
+ btree.endBulkLoad(bulkLoadCtx);
+ long end = System.currentTimeMillis();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(ins + " tuples loaded in " + (end - start) + "ms");
+ }
+
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+
+ // Build low key.
+ ArrayTupleBuilder lowKeyTb = new ArrayTupleBuilder(1);
+ ArrayTupleReference lowKey = new ArrayTupleReference();
+ TupleUtils.createIntegerTuple(lowKeyTb, lowKey, 44444);
+
+ // Build high key.
+ ArrayTupleBuilder highKeyTb = new ArrayTupleBuilder(1);
+ ArrayTupleReference highKey = new ArrayTupleReference();
+ TupleUtils.createIntegerTuple(highKeyTb, highKey, 44500);
+
+ // Prefix-Range search in [44444, 44500]
+ rangeSearch(btree, indexAccessor, fieldSerdes, lowKey, highKey);
+
+ btree.close();
+ }
+
+ private void orderedScan(BTree btree, ITreeIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes)
+ throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ordered Scan:");
+ }
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, false);
+ RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
+ indexAccessor.search(scanCursor, nullPred);
+ try {
+ while (scanCursor.hasNext()) {
+ scanCursor.next();
+ ITupleReference frameTuple = scanCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } finally {
+ scanCursor.close();
+ }
+ }
+
+ private void diskOrderScan(BTree btree, ITreeIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes)
+ throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disk-Order Scan:");
+ }
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
+ TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
+ indexAccessor.diskOrderScan(diskOrderCursor);
+ try {
+ while (diskOrderCursor.hasNext()) {
+ diskOrderCursor.next();
+ ITupleReference frameTuple = diskOrderCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } finally {
+ diskOrderCursor.close();
+ }
+ }
+
+ private void rangeSearch(BTree btree, ITreeIndexAccessor indexAccessor, ISerializerDeserializer[] fieldSerdes,
+ ITupleReference lowKey, ITupleReference highKey) throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ String lowKeyString = TupleUtils.printTuple(lowKey, fieldSerdes);
+ String highKeyString = TupleUtils.printTuple(highKey, fieldSerdes);
+ LOGGER.info("Range-Search in: [" + lowKeyString + ", " + highKeyString + "]");
+ }
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
+ MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getMultiComparator(), lowKey);
+ MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(btree.getMultiComparator(), highKey);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
+ RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, lowKeySearchCmp,
+ highKeySearchCmp);
+ indexAccessor.search(rangeCursor, rangePred);
+ try {
+ while (rangeCursor.hasNext()) {
+ rangeCursor.next();
+ ITupleReference frameTuple = rangeCursor.getTuple();
+ String rec = TupleUtils.printTuple(frameTuple, fieldSerdes);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } finally {
+ rangeCursor.close();
+ }
+ }
+
+ public static String randomString(int length, Random random) {
+ String s = Long.toHexString(Double.doubleToLongBits(random.nextDouble()));
+ StringBuilder strBuilder = new StringBuilder();
+ for (int i = 0; i < s.length() && i < length; i++) {
+ strBuilder.append(s.charAt(Math.abs(random.nextInt()) % s.length()));
+ }
+ return strBuilder.toString();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
deleted file mode 100644
index a357f2a..0000000
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ /dev/null
@@ -1,1406 +0,0 @@
-/*
- * 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;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.File;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-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.ITypeTrait;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
-import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestUtils;
-
-public class BTreeTest extends AbstractBTreeTest {
-
- private static final int PAGE_SIZE = 256;
- private static final int NUM_PAGES = 10;
- private static final int MAX_OPEN_FILES = 10;
- private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
-
- // FIXED-LENGTH KEY TEST
- // create a B-tree with one fixed-length "key" field and one fixed-length
- // "value" field
- // fill B-tree with random values using insertions (not bulk load)
- // perform ordered scan and range search
- @Test
- public void test01() throws Exception {
-
- LOGGER.info("FIXED-LENGTH KEY TEST");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
-
- // declare keys
- int keyFieldCount = 1;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- long start = System.currentTimeMillis();
-
- LOGGER.info("INSERTING INTO TREE");
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
- // 10000
- for (int i = 0; i < 10000; i++) {
-
- int f0 = rnd.nextInt() % 10000;
- int f1 = 5;
-
- tb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- ArrayTupleReference t = new ArrayTupleReference();
- t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
-
- if (i % 1000 == 0) {
- long end = System.currentTimeMillis();
- LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
- }
-
- try {
- btree.insert(t, insertOpCtx);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- // btree.printTree(leafFrame, interiorFrame);
-
- int maxPage = btree.getFreePageManager().getMaxPage(metaFrame);
- LOGGER.info("MAXPAGE: " + maxPage);
-
- long end = System.currentTimeMillis();
- long duration = end - start;
- LOGGER.info("DURATION: " + duration);
-
- // ordered scan
-
- LOGGER.info("ORDERED SCAN:");
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
- RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
- try {
- while (scanCursor.hasNext()) {
- scanCursor.next();
- ITupleReference frameTuple = scanCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- scanCursor.close();
- }
-
- // disk-order scan
- LOGGER.info("DISK-ORDER SCAN:");
- TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(leafFrame);
- BTreeOpContext diskOrderScanOpCtx = btree.createOpContext(IndexOp.DISKORDERSCAN);
- btree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
- try {
- while (diskOrderCursor.hasNext()) {
- diskOrderCursor.next();
- ITupleReference frameTuple = diskOrderCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- diskOrderCursor.close();
- }
-
- // range search in [-1000, 1000]
- LOGGER.info("RANGE SEARCH:");
-
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
-
- // build low and high keys
- ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
- DataOutput kdos = ktb.getDataOutput();
-
- ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
- IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
- keyAccessor.reset(frame);
-
- appender.reset(frame, true);
-
- // build and append low key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(-1000, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // build and append high key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(1000, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // create tuplereferences for search keys
- FrameTupleReference lowKey = new FrameTupleReference();
- lowKey.reset(keyAccessor, 0);
-
- FrameTupleReference highKey = new FrameTupleReference();
- highKey.reset(keyAccessor, 1);
-
- IBinaryComparator[] searchCmps = new IBinaryComparator[1];
- searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator searchCmp = new MultiComparator(searchCmps);
-
- RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
-
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference frameTuple = rangeCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- rangeCursor.close();
- }
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
- // COMPOSITE KEY TEST (NON-UNIQUE B-TREE)
- // create a B-tree with one two fixed-length "key" fields and one
- // fixed-length "value" field
- // fill B-tree with random values using insertions (not bulk load)
- // perform ordered scan and range search
- @Test
- public void test02() throws Exception {
-
- LOGGER.info("COMPOSITE KEY TEST");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 3;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
- typeTraits[2] = new TypeTrait(4);
-
- // declare keys
- int keyFieldCount = 2;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- long start = System.currentTimeMillis();
-
- LOGGER.info("INSERTING INTO TREE");
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
- // Magic test number: 3029. 6398. 4875.
- for (int i = 0; i < 10000; i++) {
- int f0 = rnd.nextInt() % 2000;
- int f1 = rnd.nextInt() % 1000;
- int f2 = 5;
-
- tb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f2, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1);
- }
-
- try {
- btree.insert(tuple, insertOpCtx);
- } catch (Exception e) {
- }
-
- //ISerializerDeserializer[] keySerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
- //btree.printTree(leafFrame, interiorFrame, keySerdes);
- //System.out.println("---------------------------------");
- }
-
- long end = System.currentTimeMillis();
- long duration = end - start;
- LOGGER.info("DURATION: " + duration);
-
- // try a simple index scan
- LOGGER.info("ORDERED SCAN:");
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
- RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
-
- try {
- while (scanCursor.hasNext()) {
- scanCursor.next();
- ITupleReference frameTuple = scanCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- scanCursor.close();
- }
-
- // range search in [(-3),(3)]
- LOGGER.info("RANGE SEARCH:");
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
-
- // build low and high keys
- ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
- DataOutput kdos = ktb.getDataOutput();
-
- ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
- IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
- keyAccessor.reset(frame);
-
- appender.reset(frame, true);
-
- // build and append low key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(-3, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // build and append high key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(3, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // create tuplereferences for search keys
- FrameTupleReference lowKey = new FrameTupleReference();
- lowKey.reset(keyAccessor, 0);
-
- FrameTupleReference highKey = new FrameTupleReference();
- highKey.reset(keyAccessor, 1);
-
- IBinaryComparator[] searchCmps = new IBinaryComparator[1];
- searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator searchCmp = new MultiComparator(searchCmps); // use
- // only
- // a
- // single
- // comparator
- // for
- // searching
-
- RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
-
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference frameTuple = rangeCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- rangeCursor.close();
- }
-
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
- // VARIABLE-LENGTH TEST
- // create a B-tree with one variable-length "key" field and one
- // variable-length "value" field
- // fill B-tree with random values using insertions (not bulk load)
- // perform ordered scan and range search
- @Test
- public void test03() throws Exception {
-
- LOGGER.info("VARIABLE-LENGTH KEY TEST");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-
- // declare keys
- int keyFieldCount = 1;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] recDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
- int maxLength = 10; // max string length to be generated
- for (int i = 0; i < 10000; i++) {
-
- String f0 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
- String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
-
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i);
- }
-
- try {
- btree.insert(tuple, insertOpCtx);
- } catch (Exception e) {
- }
- }
- // btree.printTree();
-
- LOGGER.info("DONE INSERTING");
-
- // ordered scan
- LOGGER.info("ORDERED SCAN:");
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
- RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
-
- try {
- while (scanCursor.hasNext()) {
- scanCursor.next();
- ITupleReference frameTuple = scanCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- scanCursor.close();
- }
-
- // range search in ["cbf", cc7"]
- LOGGER.info("RANGE SEARCH:");
-
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
-
- // build low and high keys
- ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
- DataOutput kdos = ktb.getDataOutput();
-
- ISerializerDeserializer[] keyDescSers = { UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
- IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
- keyAccessor.reset(frame);
-
- appender.reset(frame, true);
-
- // build and append low key
- ktb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize("cbf", kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // build and append high key
- ktb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize("cc7", kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // create tuplereferences for search keys
- FrameTupleReference lowKey = new FrameTupleReference();
- lowKey.reset(keyAccessor, 0);
-
- FrameTupleReference highKey = new FrameTupleReference();
- highKey.reset(keyAccessor, 1);
-
- IBinaryComparator[] searchCmps = new IBinaryComparator[1];
- searchCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator searchCmp = new MultiComparator(searchCmps);
-
- RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
-
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference frameTuple = rangeCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- rangeCursor.close();
- }
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
- // DELETION TEST
- // create a B-tree with one variable-length "key" field and one
- // variable-length "value" field
- // fill B-tree with random values using insertions, then delete entries
- // one-by-one
- // repeat procedure a few times on same B-tree
- @Test
- public void test04() throws Exception {
-
- LOGGER.info("DELETION TEST");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-
- // declare keys
- int keyFieldCount = 1;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] recDescSers = { UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
- BTreeOpContext deleteOpCtx = btree.createOpContext(IndexOp.DELETE);
-
- int runs = 3;
- for (int run = 0; run < runs; run++) {
-
- LOGGER.info("DELETION TEST RUN: " + (run + 1) + "/" + runs);
-
- LOGGER.info("INSERTING INTO BTREE");
- int maxLength = 10;
- //int ins = 16;
- int ins = 10000;
- String[] f0s = new String[ins];
- String[] f1s = new String[ins];
- int insDone = 0;
- int[] insDoneCmp = new int[ins];
- for (int i = 0; i < ins; i++) {
- String f0 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
- String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
-
- f0s[i] = f0;
- f1s[i] = f1;
-
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i);
- }
-
- try {
- btree.insert(tuple, insertOpCtx);
- insDone++;
- } catch (TreeIndexException e) {
- //e.printStackTrace();
- } catch (Exception e) {
- e.printStackTrace();
- }
-
- insDoneCmp[i] = insDone;
- }
-
- LOGGER.info("DELETING FROM BTREE");
- int delDone = 0;
- for (int i = 0; i < ins; i++) {
-
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f0s[i], dos);
- tb.addFieldEndOffset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f1s[i], dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- LOGGER.info("DELETING " + i);
- }
-
- try {
- btree.delete(tuple, deleteOpCtx);
- delDone++;
- } catch (TreeIndexException e) {
- //e.printStackTrace();
- } catch (Exception e) {
- e.printStackTrace();
- }
-
- if (insDoneCmp[i] != delDone) {
- LOGGER.info("INCONSISTENT STATE, ERROR IN DELETION TEST");
- LOGGER.info("INSDONECMP: " + insDoneCmp[i] + " " + delDone);
- break;
- }
- }
- // btree.printTree(leafFrame, interiorFrame);
-
- if (insDone != delDone) {
- LOGGER.info("ERROR! INSDONE: " + insDone + " DELDONE: " + delDone);
- break;
- }
- }
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
-
- private void orderedScan(BTree btree, IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, ISerializerDeserializer[] recDescSers) throws Exception {
- // try a simple index scan
- LOGGER.info("ORDERED SCAN:");
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
- RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
- StringBuilder scanResults = new StringBuilder();
- try {
- while (scanCursor.hasNext()) {
- scanCursor.next();
- ITupleReference frameTuple = scanCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- scanResults.append("\n" + rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- scanCursor.close();
- }
- LOGGER.info(scanResults.toString());
- }
-
- // Assuming exactly two BTree fields.
- private void compareActualAndExpected(ITreeIndexCursor actualCursor, Map<String, String> expectedValues, ISerializerDeserializer[] fieldSerdes) throws Exception {
- while (actualCursor.hasNext()) {
- actualCursor.next();
- ITupleReference tuple = actualCursor.getTuple();
- String f0 = (String) deserializeField(tuple, 0, fieldSerdes[0]);
- String f1 = (String) deserializeField(tuple, 1, fieldSerdes[1]);
- String expected = expectedValues.get(f0);
- if (!expected.equals(f1)) {
- throw new Exception("FAILED: " + f0 + " " + f1 + " " + expected);
- }
- }
- }
-
- private Object deserializeField(ITupleReference tuple, int fIdx, ISerializerDeserializer serde) throws HyracksDataException {
- DataInputStream in = new DataInputStream(new ByteArrayInputStream(tuple.getFieldData(fIdx), tuple.getFieldStart(fIdx), tuple.getFieldLength(fIdx)));
- return serde.deserialize(in);
- }
-
- // UPDATE TEST
- // create a B-tree with one variable-length "key" field and one
- // variable-length "value" field
- // fill B-tree with random values using insertions, then update entries
- // one-by-one
- // repeat procedure a few times on same B-tree
- @Test
- public void test05() throws Exception {
-
- LOGGER.info("DELETION TEST");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- typeTraits[1] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
-
- // declare keys
- int keyFieldCount = 1;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(fieldSerdes);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
- BTreeOpContext updateOpCtx = btree.createOpContext(IndexOp.UPDATE);
-
- Map<String, String> expectedValues = new HashMap<String, String>();
-
- LOGGER.info("INSERTING INTO BTREE");
- int maxLength = 10;
- int ins = 10000;
- // Only remember the keys.
- String[] f0s = new String[ins];
- for (int i = 0; i < ins; i++) {
- String f0 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
- String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
-
- f0s[i] = f0;
-
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
-
- ArrayTupleReference t = new ArrayTupleReference();
- t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- LOGGER.info("INSERTING " + i);
- }
- try {
- btree.insert(t, insertOpCtx);
- expectedValues.put(f0, f1);
- } catch (TreeIndexException e) {
- // e.printStackTrace();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- ITreeIndexCursor insertCheckCursor = new BTreeRangeSearchCursor(leafFrame);
- RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(insertCheckCursor, nullPred, searchOpCtx);
- try {
- compareActualAndExpected(insertCheckCursor, expectedValues, fieldSerdes);
- } finally {
- insertCheckCursor.close();
- }
-
- int runs = 3;
- for (int run = 0; run < runs; run++) {
-
- LOGGER.info("UPDATE TEST RUN: " + (run + 1) + "/" + runs);
-
- LOGGER.info("UPDATING BTREE");
- for (int i = 0; i < ins; i++) {
- // Generate a new random value for f1.
- String f1 = randomString(Math.abs(rnd.nextInt()) % maxLength + 1, rnd);
-
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f0s[i], dos);
- tb.addFieldEndOffset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- LOGGER.info("UPDATING " + i);
- }
-
- ArrayTupleReference t = new ArrayTupleReference();
- t.reset(tb.getFieldEndOffsets(), tb.getByteArray());
-
- try {
- btree.update(t, updateOpCtx);
- expectedValues.put(f0s[i], f1);
- } catch (TreeIndexException e) {
- e.printStackTrace();
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
-
- ITreeIndexCursor updateCheckCursor = new BTreeRangeSearchCursor(leafFrame);
- btree.search(updateCheckCursor, nullPred, searchOpCtx);
- try {
- compareActualAndExpected(updateCheckCursor, expectedValues, fieldSerdes);
- } finally {
- updateCheckCursor.close();
- }
- }
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
- // BULK LOAD TEST
- // insert 100,000 records in bulk
- // B-tree has a composite key to "simulate" non-unique index creation
- // do range search
- @Test
- public void test06() throws Exception {
-
- LOGGER.info("BULK LOAD TEST");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 3;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
- typeTraits[2] = new TypeTrait(4);
-
- // declare keys
- int keyFieldCount = 2;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
- ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- IIndexBulkLoadContext bulkLoadCtx = btree.beginBulkLoad(0.7f);
-
- // generate sorted records
- int ins = 100000;
- LOGGER.info("BULK LOADING " + ins + " RECORDS");
- long start = System.currentTimeMillis();
- for (int i = 0; i < ins; i++) {
-
- tb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(i, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(i, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(5, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- btree.bulkLoadAddTuple(tuple, bulkLoadCtx);
- }
-
- btree.endBulkLoad(bulkLoadCtx);
-
- // btree.printTree(leafFrame, interiorFrame);
-
- long end = System.currentTimeMillis();
- long duration = end - start;
- LOGGER.info("DURATION: " + duration);
-
- // range search
- LOGGER.info("RANGE SEARCH:");
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
-
- // build low and high keys
- ArrayTupleBuilder ktb = new ArrayTupleBuilder(1);
- DataOutput kdos = ktb.getDataOutput();
-
- ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
- IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
- keyAccessor.reset(frame);
-
- appender.reset(frame, true);
-
- // build and append low key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(44444, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // build and append high key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(44500, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // create tuplereferences for search keys
- FrameTupleReference lowKey = new FrameTupleReference();
- lowKey.reset(keyAccessor, 0);
-
- FrameTupleReference highKey = new FrameTupleReference();
- highKey.reset(keyAccessor, 1);
-
- IBinaryComparator[] searchCmps = new IBinaryComparator[1];
- searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator searchCmp = new MultiComparator(searchCmps);
-
- // TODO: check when searching backwards
- RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(rangeCursor, rangePred, searchOpCtx);
-
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference frameTuple = rangeCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- rangeCursor.close();
- }
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
- // TIME-INTERVAL INTERSECTION DEMO FOR EVENT PEOPLE
- // demo for Arjun to show easy support of intersection queries on
- // time-intervals
- @Test
- public void test07() throws Exception {
-
- LOGGER.info("TIME-INTERVAL INTERSECTION DEMO");
-
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
-
- // declare fields
- int fieldCount = 3;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
- typeTraits[2] = new TypeTrait(4);
-
- // declare keys
- int keyFieldCount = 2;
- IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
- cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
-
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
-
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
-
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
-
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
-
- BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
- btree.create(fileId);
- btree.open(fileId);
-
- Random rnd = new Random();
- rnd.setSeed(50);
-
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
- DataOutput dos = tb.getDataOutput();
-
- ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
-
- long start = System.currentTimeMillis();
-
- int intervalCount = 10;
- int[][] intervals = new int[intervalCount][2];
-
- intervals[0][0] = 10;
- intervals[0][1] = 20;
-
- intervals[1][0] = 11;
- intervals[1][1] = 20;
-
- intervals[2][0] = 12;
- intervals[2][1] = 20;
-
- intervals[3][0] = 13;
- intervals[3][1] = 20;
-
- intervals[4][0] = 14;
- intervals[4][1] = 20;
-
- intervals[5][0] = 20;
- intervals[5][1] = 30;
-
- intervals[6][0] = 20;
- intervals[6][1] = 31;
-
- intervals[7][0] = 20;
- intervals[7][1] = 32;
-
- intervals[8][0] = 20;
- intervals[8][1] = 33;
-
- intervals[9][0] = 20;
- intervals[9][1] = 35;
-
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
- // int exceptionCount = 0;
- for (int i = 0; i < intervalCount; i++) {
- int f0 = intervals[i][0];
- int f1 = intervals[i][1];
- int f2 = rnd.nextInt() % 100;
-
- tb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f2, dos);
- tb.addFieldEndOffset();
-
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-
- tuple.reset(accessor, 0);
-
- LOGGER.info("INSERTING " + i);
-
- try {
- btree.insert(tuple, insertOpCtx);
- } catch (Exception e) {
- }
- }
- // btree.printTree(leafFrame, interiorFrame);
- // btree.printStats();
-
- long end = System.currentTimeMillis();
- long duration = end - start;
- LOGGER.info("DURATION: " + duration);
-
- // try a simple index scan
-
- LOGGER.info("ORDERED SCAN:");
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame);
- RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(scanCursor, nullPred, searchOpCtx);
-
- try {
- while (scanCursor.hasNext()) {
- scanCursor.next();
- ITupleReference frameTuple = scanCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- // TODO: fix me.
- //print(rec + "\n");
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- scanCursor.close();
- }
-
- // try a range search
- LOGGER.info("RANGE SEARCH:");
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
-
- // build low and high keys
- ArrayTupleBuilder ktb = new ArrayTupleBuilder(cmp.getKeyFieldCount());
- DataOutput kdos = ktb.getDataOutput();
-
- ISerializerDeserializer[] keyDescSers = { IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor keyDesc = new RecordDescriptor(keyDescSers);
- IFrameTupleAccessor keyAccessor = new FrameTupleAccessor(ctx.getFrameSize(), keyDesc);
- keyAccessor.reset(frame);
-
- appender.reset(frame, true);
-
- // build and append low key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(12, kdos);
- ktb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(12, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // build and append high key
- ktb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(19, kdos);
- ktb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(19, kdos);
- ktb.addFieldEndOffset();
- appender.append(ktb.getFieldEndOffsets(), ktb.getByteArray(), 0, ktb.getSize());
-
- // create tuplereferences for search keys
- FrameTupleReference lowKey = new FrameTupleReference();
- lowKey.reset(keyAccessor, 0);
-
- FrameTupleReference highKey = new FrameTupleReference();
- highKey.reset(keyAccessor, 1);
-
- IBinaryComparator[] searchCmps = new IBinaryComparator[2];
- searchCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- searchCmps[1] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator searchCmp = new MultiComparator(searchCmps);
-
- RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, searchCmp, searchCmp);
- btree.search(rangeCursor, rangePred, searchOpCtx);
-
- try {
- while (rangeCursor.hasNext()) {
- rangeCursor.next();
- ITupleReference frameTuple = rangeCursor.getTuple();
- String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- LOGGER.info(rec);
- }
- } catch (Exception e) {
- e.printStackTrace();
- } finally {
- rangeCursor.close();
- }
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
-
- public static String randomString(int length, Random random) {
- String s = Long.toHexString(Double.doubleToLongBits(random.nextDouble()));
- StringBuilder strBuilder = new StringBuilder();
- for (int i = 0; i < s.length() && i < length; i++) {
- strBuilder.append(s.charAt(Math.abs(random.nextInt()) % s.length()));
- }
- return strBuilder.toString();
- }
-}
\ No newline at end of file
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java
index ef1e5e3..1daa273 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTestDriver.java
@@ -1,5 +1,7 @@
package edu.uci.ics.hyracks.storage.am.btree;
+import java.util.logging.Level;
+
import org.junit.Test;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -20,7 +22,9 @@
@Test
public void oneIntKeyAndValue() throws Exception {
- LOGGER.info("BTree " + getTestOpName() + " Test With One Int Key And Value.");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("BTree " + getTestOpName() + " Test With One Int Key And Value.");
+ }
ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
// Range search in [-1000, 1000]
@@ -32,8 +36,10 @@
}
@Test
- public void twoIntKeys() throws Exception {
- LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys.");
+ public void twoIntKeys() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys.");
+ }
ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -50,8 +56,10 @@
}
@Test
- public void twoIntKeysAndValues() throws Exception {
- LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys And Values.");
+ public void twoIntKeysAndValues() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("BTree " + getTestOpName() + " Test With Two Int Keys And Values.");
+ }
ISerializerDeserializer[] fieldSerdes = { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
@@ -69,7 +77,9 @@
@Test
public void oneStringKeyAndValue() throws Exception {
- LOGGER.info("BTree " + getTestOpName() + " Test With One String Key And Value.");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("BTree " + getTestOpName() + " Test With One String Key And Value.");
+ }
ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
@@ -82,8 +92,10 @@
}
@Test
- public void twoStringKeys() throws Exception {
- LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys.");
+ public void twoStringKeys() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys.");
+ }
ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
@@ -100,8 +112,10 @@
}
@Test
- public void twoStringKeysAndValues() throws Exception {
- LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys And Values.");
+ public void twoStringKeysAndValues() throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("BTree " + getTestOpName() + " Test With Two String Keys And Values.");
+ }
ISerializerDeserializer[] fieldSerdes = { UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
similarity index 92%
rename from hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
rename to hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index 0f3edd6..ac85afd 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -18,6 +18,7 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
import java.util.Random;
+import java.util.logging.Level;
import org.junit.Assert;
import org.junit.Test;
@@ -47,7 +48,7 @@
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
-public class BTreeFieldPrefixNSMTest extends AbstractBTreeTest {
+public class FieldPrefixNSMTest extends AbstractBTreeTest {
private static final int PAGE_SIZE = 32768; // 32K
private static final int NUM_PAGES = 40;
@@ -56,8 +57,11 @@
private ITupleReference createTuple(IHyracksTaskContext ctx, int f0, int f1, int f2, boolean print)
throws HyracksDataException {
- if (print)
- LOGGER.info("CREATING: " + f0 + " " + f1 + " " + f2);
+ if (print) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("CREATING: " + f0 + " " + f1 + " " + f2);
+ }
+ }
ByteBuffer buf = ctx.allocateFrame();
FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -135,8 +139,11 @@
// insert records with random calls to compact and compress
for (int i = 0; i < numRecords; i++) {
- if ((i + 1) % 100 == 0)
- LOGGER.info("INSERTING " + (i + 1) + " / " + numRecords);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if ((i + 1) % 100 == 0) {
+ LOGGER.info("INSERTING " + (i + 1) + " / " + numRecords);
+ }
+ }
int a = rnd.nextInt() % smallMax;
int b = rnd.nextInt() % smallMax;
@@ -174,9 +181,11 @@
// delete records with random calls to compact and compress
for (int i = 0; i < numRecords; i++) {
-
- if ((i + 1) % 100 == 0)
- LOGGER.info("DELETING " + (i + 1) + " / " + numRecords);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if ((i + 1) % 100 == 0) {
+ LOGGER.info("DELETING " + (i + 1) + " / " + numRecords);
+ }
+ }
ITupleReference tuple = createTuple(ctx, savedFields[i][0], savedFields[i][1], savedFields[i][2], false);
try {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 14246f7..922c715 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -22,6 +22,7 @@
import java.util.Collections;
import java.util.Random;
import java.util.TreeSet;
+import java.util.logging.Level;
import org.junit.Before;
import org.junit.Test;
@@ -43,21 +44,20 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
public class RangeSearchCursorTest extends AbstractBTreeTest {
// Declare fields
@@ -80,14 +80,16 @@
@Test
public void uniqueIndexTest() throws Exception {
- LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("TESTING RANGE SEARCH CURSOR ON UNIQUE INDEX");
+ }
// declare keys
int keyFieldCount = 1;
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
tupleWriterFactory);
@@ -106,7 +108,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// generate keys
int numKeys = 50;
@@ -128,7 +130,7 @@
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (BTreeException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -165,7 +167,9 @@
@Test
public void nonUniqueIndexTest() throws Exception {
- LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE INDEX");
+ }
// declare keys
int keyFieldCount = 2;
@@ -173,7 +177,7 @@
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
tupleWriterFactory);
@@ -192,7 +196,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// generate keys
int numKeys = 50;
@@ -211,7 +215,7 @@
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (BTreeException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -248,7 +252,9 @@
@Test
public void nonUniqueFieldPrefixIndexTest() throws Exception {
- LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("TESTING RANGE SEARCH CURSOR ON NONUNIQUE FIELD-PREFIX COMPRESSED INDEX");
+ }
// declare keys
int keyFieldCount = 2;
@@ -256,7 +262,7 @@
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE ;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
tupleWriterFactory);
@@ -275,7 +281,7 @@
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// generate keys
int numKeys = 50;
@@ -294,7 +300,7 @@
tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (BTreeException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -405,12 +411,12 @@
int lowKey = i;
int highKey = j;
- ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame);
+ ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor(leafFrame, false);
RangePredicate rangePred = createRangePredicate(lowKey,
highKey, isForward, lowKeyInclusive, highKeyInclusive,
btree.getMultiComparator());
- BTreeOpContext searchOpCtx = btree.createOpContext(IndexOp.SEARCH);
- btree.search(rangeCursor, rangePred, searchOpCtx);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+ indexAccessor.search(rangeCursor, rangePred);
try {
while (rangeCursor.hasNext()) {
@@ -448,32 +454,35 @@
else
u = ')';
- LOGGER.info("RANGE: " + l + " " + lowKey + " , "
- + highKey + " " + u);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("RANGE: " + l + " " + lowKey + " , " + highKey + " " + u);
+ }
StringBuilder strBuilder = new StringBuilder();
for (Integer r : expectedResults) {
strBuilder.append(r + " ");
}
- LOGGER.info(strBuilder.toString());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(strBuilder.toString());
+ }
}
}
if (results.size() == expectedResults.size()) {
for (int k = 0; k < results.size(); k++) {
if (!results.get(k).equals(expectedResults.get(k))) {
- LOGGER.info("DIFFERENT RESULTS AT: i=" + i
- + " j=" + j + " k=" + k);
- LOGGER.info(results.get(k) + " "
- + expectedResults.get(k));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DIFFERENT RESULTS AT: i=" + i + " j=" + j + " k=" + k);
+ LOGGER.info(results.get(k) + " " + expectedResults.get(k));
+ }
return false;
}
}
} else {
- LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i
- + " j=" + j);
- LOGGER.info("RESULTS: " + results.size());
- LOGGER.info("EXPECTED RESULTS: "
- + expectedResults.size());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("UNEQUAL NUMBER OF RESULTS AT: i=" + i + " j=" + j);
+ LOGGER.info("RESULTS: " + results.size());
+ LOGGER.info("EXPECTED RESULTS: " + expectedResults.size());
+ }
return false;
}
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
similarity index 89%
rename from hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
rename to hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
index 7c181c8..dc524d7 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StatsTest.java
@@ -4,6 +4,7 @@
import java.io.File;
import java.nio.ByteBuffer;
import java.util.Random;
+import java.util.logging.Level;
import org.junit.Test;
@@ -26,19 +27,18 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
-import edu.uci.ics.hyracks.storage.am.btree.util.BTreeUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexBufferCacheWarmup;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
@@ -47,7 +47,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestUtils;
-public class BTreeStatsTest extends AbstractBTreeTest {
+public class StatsTest extends AbstractBTreeTest {
// private static final int PAGE_SIZE = 256;
// private static final int NUM_PAGES = 10;
@@ -80,7 +80,7 @@
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -102,7 +102,9 @@
long start = System.currentTimeMillis();
- LOGGER.info("INSERTING INTO TREE");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("INSERTING INTO TREE");
+ }
ByteBuffer frame = ctx.allocateFrame();
FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -116,8 +118,7 @@
accessor.reset(frame);
FrameTupleReference tuple = new FrameTupleReference();
- BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT);
-
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
// 10000
for (int i = 0; i < 100000; i++) {
@@ -135,13 +136,15 @@
tuple.reset(accessor, 0);
- if (i % 10000 == 0) {
- long end = System.currentTimeMillis();
- LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 10000 == 0) {
+ long end = System.currentTimeMillis();
+ LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+ }
}
try {
- btree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -151,7 +154,9 @@
TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
btree.getRootPageId());
TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
- LOGGER.info("\n" + stats.toString());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("\n" + stats.toString());
+ }
TreeIndexBufferCacheWarmup bufferCacheWarmup = new TreeIndexBufferCacheWarmup(bufferCache, freePageManager,
fileId);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index 7e4b2e3..ac4133d 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -19,6 +19,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
+import java.util.logging.Level;
import org.junit.Test;
@@ -85,7 +86,9 @@
private void pinRandomPage() {
int pageId = Math.abs(rnd.nextInt() % maxPages);
- LOGGER.info(workerId + " PINNING PAGE: " + pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " PINNING PAGE: " + pageId);
+ }
try {
ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
@@ -99,14 +102,18 @@
break;
case FTA_READONLY: {
- LOGGER.info(workerId + " S LATCHING: " + pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " S LATCHING: " + pageId);
+ }
page.acquireReadLatch();
latch = LatchType.LATCH_S;
}
break;
case FTA_WRITEONLY: {
- LOGGER.info(workerId + " X LATCHING: " + pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " X LATCHING: " + pageId);
+ }
page.acquireWriteLatch();
latch = LatchType.LATCH_X;
}
@@ -114,11 +121,15 @@
case FTA_MIXED: {
if (rnd.nextInt() % 2 == 0) {
- LOGGER.info(workerId + " S LATCHING: " + pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " S LATCHING: " + pageId);
+ }
page.acquireReadLatch();
latch = LatchType.LATCH_S;
} else {
- LOGGER.info(workerId + " X LATCHING: " + pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " X LATCHING: " + pageId);
+ }
page.acquireWriteLatch();
latch = LatchType.LATCH_X;
}
@@ -141,14 +152,20 @@
if (plPage.latch != null) {
if (plPage.latch == LatchType.LATCH_S) {
- LOGGER.info(workerId + " S UNLATCHING: " + plPage.pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " S UNLATCHING: " + plPage.pageId);
+ }
plPage.page.releaseReadLatch();
} else {
- LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
+ }
plPage.page.releaseWriteLatch();
}
}
- LOGGER.info(workerId + " UNPINNING PAGE: " + plPage.pageId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " UNPINNING PAGE: " + plPage.pageId);
+ }
bufferCache.unpin(plPage.page);
pinnedPages.remove(index);
@@ -158,7 +175,9 @@
}
private void openFile() {
- LOGGER.info(workerId + " OPENING FILE: " + fileId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " OPENING FILE: " + fileId);
+ }
try {
bufferCache.openFile(fileId);
fileIsOpen = true;
@@ -168,7 +187,9 @@
}
private void closeFile() {
- LOGGER.info(workerId + " CLOSING FILE: " + fileId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " CLOSING FILE: " + fileId);
+ }
try {
bufferCache.closeFile(fileId);
fileIsOpen = false;
@@ -185,7 +206,9 @@
while (loopCount < maxLoopCount) {
loopCount++;
- LOGGER.info(workerId + " LOOP: " + loopCount + "/" + maxLoopCount);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(workerId + " LOOP: " + loopCount + "/" + maxLoopCount);
+ }
if (fileIsOpen) {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
new file mode 100644
index 0000000..7a61adc
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/UpdateSearchTest.java
@@ -0,0 +1,152 @@
+package edu.uci.ics.hyracks.storage.am.btree;
+
+import java.util.Random;
+import java.util.logging.Level;
+
+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.ITypeTrait;
+import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
+import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
+import edu.uci.ics.hyracks.storage.am.btree.util.AbstractBTreeTest;
+import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
+
+public class UpdateSearchTest extends AbstractBTreeTest {
+
+ // Update scan test on fixed-length tuples.
+ @Test
+ public void test01() throws Exception {
+ // declare fields
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = new TypeTrait(4);
+ typeTraits[1] = new TypeTrait(4);
+
+ // declare keys
+ int keyFieldCount = 1;
+ IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
+ cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
+
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
+
+ ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
+ BTree btree = new BTree(bufferCache, fieldCount, cmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree.create(btreeFileId);
+ btree.open(btreeFileId);
+
+ Random rnd = new Random();
+ rnd.setSeed(50);
+
+ long start = System.currentTimeMillis();
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("INSERTING INTO TREE");
+ }
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
+ ArrayTupleReference insertTuple = new ArrayTupleReference();
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
+
+ int numInserts = 10000;
+ for (int i = 0; i < 10000; i++) {
+ int f0 = rnd.nextInt() % 10000;
+ int f1 = 5;
+ TupleUtils.createIntegerTuple(tb, insertTuple, f0, f1);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 10000 == 0) {
+ long end = System.currentTimeMillis();
+ LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+ }
+ }
+
+ try {
+ indexAccessor.insert(insertTuple);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ long end = System.currentTimeMillis();
+ long duration = end - start;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DURATION: " + duration);
+ }
+
+ // Update scan.
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("UPDATE SCAN:");
+ }
+ // Set the cursor to X latch nodes.
+ ITreeIndexCursor updateScanCursor = new BTreeRangeSearchCursor(leafFrame, true);
+ RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
+ indexAccessor.search(updateScanCursor, nullPred);
+ try {
+ while (updateScanCursor.hasNext()) {
+ updateScanCursor.next();
+ ITupleReference tuple = updateScanCursor.getTuple();
+ // Change the value field.
+ IntegerSerializerDeserializer.putInt(10, tuple.getFieldData(1), tuple.getFieldStart(1));
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ updateScanCursor.close();
+ }
+
+ // Ordered scan to verify the values.
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("ORDERED SCAN:");
+ }
+ // Set the cursor to X latch nodes.
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(leafFrame, true);
+ indexAccessor.search(scanCursor, nullPred);
+ try {
+ while (scanCursor.hasNext()) {
+ scanCursor.next();
+ ITupleReference tuple = scanCursor.getTuple();
+ String rec = TupleUtils.printTuple(tuple, recDescSers);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ scanCursor.close();
+ }
+ btree.close();
+ }
+}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
index dff1e9c..9630a1b 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/AbstractBTreeTest.java
@@ -34,7 +34,7 @@
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
protected final static String tmpDir = System.getProperty("java.io.tmpdir");
protected final static String sep = System.getProperty("file.separator");
- protected String fileName;
+ protected String fileName;
@Before
public void setUp() throws HyracksDataException {
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
index c56f1b6..f1b03c1 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestContext.java
@@ -23,39 +23,40 @@
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@SuppressWarnings("rawtypes")
-public final class BTreeTestContext {
+public final class BTreeTestContext {
public final ISerializerDeserializer[] fieldSerdes;
public final IBufferCache bufferCache;
public final BTree btree;
public final IBTreeLeafFrame leafFrame;
public final IBTreeInteriorFrame interiorFrame;
- public final ITreeIndexMetaDataFrame metaFrame;
+ public final ITreeIndexMetaDataFrame metaFrame;
public final ArrayTupleBuilder tupleBuilder;
public final ArrayTupleReference tuple = new ArrayTupleReference();
public final TreeSet<CheckTuple> checkTuples = new TreeSet<CheckTuple>();
- public final BTreeOpContext opCtx;
-
- public BTreeTestContext(IBufferCache bufferCache, ISerializerDeserializer[] fieldSerdes, BTree btree, IBTreeLeafFrame leafFrame,
- IBTreeInteriorFrame interiorFrame, ITreeIndexMetaDataFrame metaFrame, BTreeOpContext opCtx) {
+ public final ITreeIndexAccessor indexAccessor;
+
+ public BTreeTestContext(IBufferCache bufferCache, ISerializerDeserializer[] fieldSerdes, BTree btree,
+ IBTreeLeafFrame leafFrame, IBTreeInteriorFrame interiorFrame, ITreeIndexMetaDataFrame metaFrame,
+ ITreeIndexAccessor indexAccessor) {
this.bufferCache = bufferCache;
this.fieldSerdes = fieldSerdes;
this.btree = btree;
this.leafFrame = leafFrame;
this.interiorFrame = interiorFrame;
this.metaFrame = metaFrame;
- this.opCtx = opCtx;
+ this.indexAccessor = indexAccessor;
this.tupleBuilder = new ArrayTupleBuilder(fieldSerdes.length);
}
-
+
public int getFieldCount() {
return fieldSerdes.length;
}
-
+
public int getKeyFieldCount() {
return btree.getMultiComparator().getKeyFieldCount();
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
index 8e7a43b..ca543a2 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/util/BTreeTestUtils.java
@@ -10,6 +10,7 @@
import java.util.NavigableSet;
import java.util.Random;
import java.util.TreeSet;
+import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -28,15 +29,15 @@
import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
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.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -51,13 +52,12 @@
BTree btree = BTreeUtils.createBTree(bufferCache, btreeFileId, typeTraits, cmps, leafType);
btree.create(btreeFileId);
btree.open(btreeFileId);
- // Set an arbitrary index op in the context. Will be reset anyway.
- BTreeOpContext opCtx = btree.createOpContext(IndexOp.SEARCH);
+ ITreeIndexAccessor indexAccessor = btree.createAccessor();
IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame();
IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) btree.getInteriorFrameFactory().createFrame();
ITreeIndexMetaDataFrame metaFrame = btree.getFreePageManager().getMetaDataFrameFactory().createFrame();
- BTreeTestContext testCtx = new BTreeTestContext(bufferCache, fieldSerdes, btree, leafFrame, interiorFrame, metaFrame, opCtx);
+ BTreeTestContext testCtx = new BTreeTestContext(bufferCache, fieldSerdes, btree, leafFrame, interiorFrame, metaFrame, indexAccessor);
return testCtx;
}
@@ -77,8 +77,8 @@
@SuppressWarnings("unchecked")
private static CheckTuple createCheckTupleFromTuple(ITupleReference tuple, ISerializerDeserializer[] fieldSerdes, int numKeys) throws HyracksDataException {
CheckTuple checkTuple = new CheckTuple(fieldSerdes.length, numKeys);
- int numFields = Math.min(fieldSerdes.length, tuple.getFieldCount());
- for (int i = 0; i < numFields; i++) {
+ int fieldCount = Math.min(fieldSerdes.length, tuple.getFieldCount());
+ for (int i = 0; i < fieldCount; i++) {
ByteArrayInputStream inStream = new ByteArrayInputStream(
tuple.getFieldData(i), tuple.getFieldStart(i),
tuple.getFieldLength(i));
@@ -102,11 +102,12 @@
}
public static void checkOrderedScan(BTreeTestContext testCtx) throws Exception {
- LOGGER.info("Testing Ordered Scan.");
- ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Testing Ordered Scan.");
+ }
+ ITreeIndexCursor scanCursor = new BTreeRangeSearchCursor(testCtx.leafFrame, false);
RangePredicate nullPred = new RangePredicate(true, null, null, true, true, null, null);
- testCtx.opCtx.reset(IndexOp.SEARCH);
- testCtx.btree.search(scanCursor, nullPred, testCtx.opCtx);
+ testCtx.indexAccessor.search(scanCursor, nullPred);
Iterator<CheckTuple> checkIter = testCtx.checkTuples.iterator();
int actualCount = 0;
try {
@@ -129,10 +130,11 @@
}
public static void checkDiskOrderScan(BTreeTestContext testCtx) throws Exception {
- LOGGER.info("Testing Disk-Order Scan.");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Testing Disk-Order Scan.");
+ }
ITreeIndexCursor diskOrderCursor = new TreeDiskOrderScanCursor(testCtx.leafFrame);
- testCtx.opCtx.reset(IndexOp.DISKORDERSCAN);
- testCtx.btree.diskOrderScan(diskOrderCursor, testCtx.opCtx);
+ testCtx.indexAccessor.diskOrderScan(diskOrderCursor);
int actualCount = 0;
try {
while (diskOrderCursor.hasNext()) {
@@ -156,13 +158,14 @@
}
public static void checkRangeSearch(BTreeTestContext testCtx, ITupleReference lowKey, ITupleReference highKey, boolean lowKeyInclusive, boolean highKeyInclusive) throws Exception {
- LOGGER.info("Testing Range Search.");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Testing Range Search.");
+ }
MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), lowKey);
MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(testCtx.btree.getMultiComparator(), highKey);
- ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
+ ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame, false);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp);
- testCtx.opCtx.reset(IndexOp.SEARCH);
- testCtx.btree.search(searchCursor, rangePred, testCtx.opCtx);
+ testCtx.indexAccessor.search(searchCursor, rangePred);
// Get the subset of elements from the expected set within given key range.
CheckTuple lowKeyCheck = createCheckTupleFromTuple(lowKey, testCtx.fieldSerdes, lowKeyCmp.getKeyFieldCount());
CheckTuple highKeyCheck = createCheckTupleFromTuple(highKey, testCtx.fieldSerdes, highKeyCmp.getKeyFieldCount());
@@ -197,14 +200,15 @@
}
public static void checkPointSearches(BTreeTestContext testCtx) throws Exception {
- LOGGER.info("Testing Point Searches On All Expected Keys.");
- ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Testing Point Searches On All Expected Keys.");
+ }
+ ITreeIndexCursor searchCursor = new BTreeRangeSearchCursor(testCtx.leafFrame, false);
ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
ArrayTupleReference lowKey = new ArrayTupleReference();
ArrayTupleBuilder highKeyBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
ArrayTupleReference highKey = new ArrayTupleReference();
- testCtx.opCtx.reset(IndexOp.SEARCH);
RangePredicate rangePred = new RangePredicate(true, lowKey, highKey, true, true, null, null);
// Iterate through expected tuples, and perform a point search in the BTree to verify the tuple can be reached.
@@ -219,7 +223,7 @@
rangePred.setLowKeyComparator(lowKeyCmp);
rangePred.setHighKeyComparator(highKeyCmp);
- testCtx.btree.search(searchCursor, rangePred, testCtx.opCtx);
+ testCtx.indexAccessor.search(searchCursor, rangePred);
try {
// We expect exactly one answer.
@@ -267,11 +271,8 @@
}
public static void insertIntTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
- int numFields = testCtx.getFieldCount();
+ int fieldCount = testCtx.getFieldCount();
int numKeyFields = testCtx.getKeyFieldCount();
-
- testCtx.opCtx.reset(IndexOp.INSERT);
-
int[] tupleValues = new int[testCtx.getFieldCount()];
// Scale range of values according to number of keys.
// For example, for 2 keys we want the square root of numTuples, for 3 keys the cube root of numTuples, etc.
@@ -282,17 +283,19 @@
tupleValues[j] = rnd.nextInt() % maxValue;
}
// Set values.
- for (int j = numKeyFields; j < numFields; j++) {
+ for (int j = numKeyFields; j < fieldCount; j++) {
tupleValues[j] = j;
}
TupleUtils.createIntegerTuple(testCtx.tupleBuilder, testCtx.tuple, tupleValues);
- if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
- LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+ LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+ }
}
try {
- testCtx.btree.insert(testCtx.tuple, testCtx.opCtx);
+ testCtx.indexAccessor.insert(testCtx.tuple);
// Set expected values. Do this only after insertion succeeds because we ignore duplicate keys.
- CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(numFields, numKeyFields);
+ CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldCount, numKeyFields);
for(int v : tupleValues) {
checkTuple.add(v);
}
@@ -304,15 +307,14 @@
}
public static void insertStringTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
- int numFields = testCtx.getFieldCount();
+ int fieldCount = testCtx.getFieldCount();
int numKeyFields = testCtx.getKeyFieldCount();
-
- testCtx.opCtx.reset(IndexOp.INSERT);
-
- Object[] tupleValues = new Object[numFields];
+ Object[] tupleValues = new Object[fieldCount];
for (int i = 0; i < numTuples; i++) {
- if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
- LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+ LOGGER.info("Inserting Tuple " + (i + 1) + "/" + numTuples);
+ }
}
// Set keys.
for (int j = 0; j < numKeyFields; j++) {
@@ -320,14 +322,14 @@
tupleValues[j] = getRandomString(length, rnd);
}
// Set values.
- for (int j = numKeyFields; j < numFields; j++) {
+ for (int j = numKeyFields; j < fieldCount; j++) {
tupleValues[j] = getRandomString(5, rnd);
}
TupleUtils.createTuple(testCtx.tupleBuilder, testCtx.tuple, testCtx.fieldSerdes, tupleValues);
try {
- testCtx.btree.insert(testCtx.tuple, testCtx.opCtx);
+ testCtx.indexAccessor.insert(testCtx.tuple);
// Set expected values. Do this only after insertion succeeds because we ignore duplicate keys.
- CheckTuple<String> checkTuple = new CheckTuple<String>(numFields, numKeyFields);
+ CheckTuple<String> checkTuple = new CheckTuple<String>(fieldCount, numKeyFields);
for(Object v : tupleValues) {
checkTuple.add((String)v);
}
@@ -339,7 +341,7 @@
}
public static void bulkLoadIntTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
- int numFields = testCtx.getFieldCount();
+ int fieldCount = testCtx.getFieldCount();
int numKeyFields = testCtx.getKeyFieldCount();
int[] tupleValues = new int[testCtx.getFieldCount()];
int maxValue = (int)Math.ceil(Math.pow(numTuples, 1.0/(double)numKeyFields));
@@ -349,12 +351,12 @@
tupleValues[j] = rnd.nextInt() % maxValue;
}
// Set values.
- for (int j = numKeyFields; j < numFields; j++) {
+ for (int j = numKeyFields; j < fieldCount; j++) {
tupleValues[j] = j;
}
// Set expected values. We also use these as the pre-sorted stream for bulk loading.
- CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(numFields, numKeyFields);
+ CheckTuple<Integer> checkTuple = new CheckTuple<Integer>(fieldCount, numKeyFields);
for(int v : tupleValues) {
checkTuple.add(v);
}
@@ -365,9 +367,9 @@
}
public static void bulkLoadStringTuples(BTreeTestContext testCtx, int numTuples, Random rnd) throws Exception {
- int numFields = testCtx.getFieldCount();
+ int fieldCount = testCtx.getFieldCount();
int numKeyFields = testCtx.getKeyFieldCount();
- String[] tupleValues = new String[numFields];
+ String[] tupleValues = new String[fieldCount];
for (int i = 0; i < numTuples; i++) {
// Set keys.
for (int j = 0; j < numKeyFields; j++) {
@@ -375,11 +377,11 @@
tupleValues[j] = getRandomString(length, rnd);
}
// Set values.
- for (int j = numKeyFields; j < numFields; j++) {
+ for (int j = numKeyFields; j < fieldCount; j++) {
tupleValues[j] = getRandomString(5, rnd);
}
// Set expected values. We also use these as the pre-sorted stream for bulk loading.
- CheckTuple<String> checkTuple = new CheckTuple<String>(numFields, numKeyFields);
+ CheckTuple<String> checkTuple = new CheckTuple<String>(fieldCount, numKeyFields);
for(String v : tupleValues) {
checkTuple.add(v);
}
@@ -389,16 +391,18 @@
bulkLoadCheckTuples(testCtx, numTuples);
}
- private static void bulkLoadCheckTuples(BTreeTestContext testCtx, int numTuples) throws HyracksDataException, TreeIndexException {
- int numFields = testCtx.getFieldCount();
- ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(numFields);
+ private static void bulkLoadCheckTuples(BTreeTestContext testCtx, int numTuples) throws HyracksDataException, TreeIndexException, PageAllocationException {
+ int fieldCount = testCtx.getFieldCount();
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference tuple = new ArrayTupleReference();
// Perform bulk load.
IIndexBulkLoadContext bulkLoadCtx = testCtx.btree.beginBulkLoad(0.7f);
int c = 1;
for (CheckTuple checkTuple : testCtx.checkTuples) {
- if (c % (numTuples / 10) == 0) {
- LOGGER.info("Bulk Loading Tuple " + c + "/" + numTuples);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (c % (numTuples / 10) == 0) {
+ LOGGER.info("Bulk Loading Tuple " + c + "/" + numTuples);
+ }
}
createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, testCtx.fieldSerdes);
testCtx.btree.bulkLoadAddTuple(tuple, bulkLoadCtx);
@@ -411,7 +415,6 @@
ArrayTupleBuilder deleteTupleBuilder = new ArrayTupleBuilder(testCtx.btree.getMultiComparator().getKeyFieldCount());
ArrayTupleReference deleteTuple = new ArrayTupleReference();
int numCheckTuples = testCtx.checkTuples.size();
- testCtx.opCtx.reset(IndexOp.DELETE);
// Copy CheckTuple references into array, so we can randomly pick from there.
CheckTuple[] checkTuples = new CheckTuple[numCheckTuples];
int idx = 0;
@@ -419,13 +422,15 @@
checkTuples[idx++] = checkTuple;
}
for (int i = 0; i < numTuples && numCheckTuples > 0; i++) {
- if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
- LOGGER.info("Deleting Tuple " + (i + 1) + "/" + numTuples);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+ LOGGER.info("Deleting Tuple " + (i + 1) + "/" + numTuples);
+ }
}
int checkTupleIdx = Math.abs(rnd.nextInt() % numCheckTuples);
CheckTuple checkTuple = checkTuples[checkTupleIdx];
createTupleFromCheckTuple(checkTuple, deleteTupleBuilder, deleteTuple, testCtx.fieldSerdes);
- testCtx.btree.delete(deleteTuple, testCtx.opCtx);
+ testCtx.indexAccessor.delete(deleteTuple);
// Remove check tuple from expected results.
testCtx.checkTuples.remove(checkTuple);
@@ -449,7 +454,6 @@
ArrayTupleBuilder updateTupleBuilder = new ArrayTupleBuilder(fieldCount);
ArrayTupleReference updateTuple = new ArrayTupleReference();
int numCheckTuples = testCtx.checkTuples.size();
- testCtx.opCtx.reset(IndexOp.UPDATE);
// Copy CheckTuple references into array, so we can randomly pick from there.
CheckTuple[] checkTuples = new CheckTuple[numCheckTuples];
int idx = 0;
@@ -457,8 +461,10 @@
checkTuples[idx++] = checkTuple;
}
for (int i = 0; i < numTuples && numCheckTuples > 0; i++) {
- if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
- LOGGER.info("Updating Tuple " + (i + 1) + "/" + numTuples);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if ((i + 1) % (numTuples / Math.min(10, numTuples)) == 0) {
+ LOGGER.info("Updating Tuple " + (i + 1) + "/" + numTuples);
+ }
}
int checkTupleIdx = Math.abs(rnd.nextInt() % numCheckTuples);
CheckTuple checkTuple = checkTuples[checkTupleIdx];
@@ -469,7 +475,7 @@
}
createTupleFromCheckTuple(checkTuple, updateTupleBuilder, updateTuple, testCtx.fieldSerdes);
- testCtx.btree.update(updateTuple, testCtx.opCtx);
+ testCtx.indexAccessor.update(updateTuple);
// Swap with last "valid" CheckTuple.
CheckTuple tmp = checkTuples[numCheckTuples - 1];
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index fa1833a..1f1cc25 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -1,3 +1,18 @@
+/*
+ * 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.invertedindex;
import java.io.DataOutput;
@@ -43,6 +58,7 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.TOccurrenceSearcher;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.invertedindex.util.InvertedIndexUtils;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
@@ -64,9 +80,9 @@
protected FileReference btreeFile = new FileReference(new File(btreeFileName));
protected int btreeFileId;
- // declare btree fields
- protected int fieldCount = 5;
- protected ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ // declare token type traits
+ protected ITypeTrait[] tokenTypeTraits = new ITypeTrait[] { ITypeTrait.VARLEN_TYPE_TRAIT };
+ protected ITypeTrait[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
// declare btree keys
protected int btreeKeyFieldCount = 1;
@@ -74,7 +90,7 @@
protected MultiComparator btreeCmp = new MultiComparator(btreeBinCmps);
// btree frame factories
- protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
protected ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
protected ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
protected ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
@@ -153,22 +169,11 @@
btreeFileId = fmp.lookupFileId(btreeFile);
bufferCache.openFile(btreeFileId);
- // token (key)
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- // startPageId
- typeTraits[1] = new TypeTrait(4);
- // endPageId
- typeTraits[2] = new TypeTrait(4);
- // startOff
- typeTraits[3] = new TypeTrait(4);
- // numElements
- typeTraits[4] = new TypeTrait(4);
-
btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- btree = new BTree(bufferCache, fieldCount, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
btree.create(btreeFileId);
btree.open(btreeFileId);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index 86b3107..bbbb743 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -48,18 +48,16 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -67,6 +65,7 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.FixedSizeElementInvertedListCursor;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+import edu.uci.ics.hyracks.storage.am.invertedindex.util.InvertedIndexUtils;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
@@ -104,19 +103,9 @@
int invListsFileId = fmp.lookupFileId(invListsFile);
bufferCache.openFile(invListsFileId);
- // declare btree fields
- int fieldCount = 5;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- // token (key)
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- // startPageId
- typeTraits[1] = new TypeTrait(4);
- // endPageId
- typeTraits[2] = new TypeTrait(4);
- // startOff
- typeTraits[3] = new TypeTrait(4);
- // numElements
- typeTraits[4] = new TypeTrait(4);
+ // Declare token type traits, and compute BTree type traits.
+ ITypeTrait[] tokenTypeTraits = new ITypeTrait[] { ITypeTrait.VARLEN_TYPE_TRAIT };
+ ITypeTrait[] btreeTypeTraits = InvertedIndexUtils.getBTreeTypeTraits(tokenTypeTraits);
// declare btree keys
int keyFieldCount = 1;
@@ -125,18 +114,16 @@
MultiComparator btreeCmp = new MultiComparator(cmps);
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
- ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- BTree btree = new BTree(bufferCache, fieldCount, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
+ BTree btree = new BTree(bufferCache, btreeTypeTraits.length, btreeCmp, freePageManager, interiorFrameFactory, leafFrameFactory);
btree.create(btreeFileId);
btree.open(btreeFileId);
@@ -226,7 +213,7 @@
// ------- START VERIFICATION -----------
- ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+ ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
FrameTupleReference searchKey = new FrameTupleReference();
RangePredicate btreePred = new RangePredicate(true, searchKey, searchKey, true, true, btreeCmp, btreeCmp);
@@ -241,7 +228,7 @@
IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
tokenAccessor.reset(frame);
- BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH);
+ ITreeIndexAccessor btreeAccessor = invIndex.getBTree().createAccessor();
// verify created inverted lists one-by-one
for (int i = 0; i < tokens.size(); i++) {
@@ -256,7 +243,7 @@
searchKey.reset(tokenAccessor, 0);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
+ invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
invListCursor.pinPagesSync();
int checkIndex = 0;
@@ -291,7 +278,7 @@
searchKey.reset(tokenAccessor, 0);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
+ invIndex.openCursor(btreeCursor, btreePred, btreeAccessor, invListCursor);
invListCursor.pinPagesSync();
Assert.assertEquals(invListCursor.hasNext(), false);
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
index e6dba19..1299fcb 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
@@ -17,6 +17,7 @@
import java.util.ArrayList;
import java.util.List;
+import java.util.logging.Level;
import org.junit.Assert;
import org.junit.Before;
@@ -27,6 +28,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -66,7 +68,7 @@
loadData();
}
- public void loadData() throws HyracksDataException, TreeIndexException {
+ public void loadData() throws HyracksDataException, TreeIndexException, PageAllocationException {
tokens.add("compilers");
tokens.add("computer");
tokens.add("databases");
@@ -212,7 +214,10 @@
totalTime += timeEnd - timeStart;
}
double avgTime = totalTime / (double) repeats;
- LOGGER.info(i + ": " + "\"" + queryString + "\": " + avgTime + "ms");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(i + ": " + "\"" + queryString + "\": " + avgTime
+ + "ms");
+ }
if (!panic) {
@@ -233,12 +238,16 @@
}
if (expectedResults.size() != checkIndex) {
- LOGGER.info("CHECKING");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("CHECKING");
+ }
StringBuilder expectedStrBuilder = new StringBuilder();
for (Integer x : expectedResults) {
expectedStrBuilder.append(x + " ");
}
- LOGGER.info(expectedStrBuilder.toString());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(expectedStrBuilder.toString());
+ }
}
Assert.assertEquals(expectedResults.size(), checkIndex);
@@ -266,27 +275,39 @@
public void jaccardKeywordQueryTest() throws Exception {
JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
- LOGGER.info("JACCARD: " + 1.0f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 1.0f);
+ }
searchModifier.setJaccThresh(1.0f);
runQueries(searchModifier, 50);
- LOGGER.info("JACCARD: " + 0.9f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.9f);
+ }
searchModifier.setJaccThresh(0.9f);
runQueries(searchModifier, 50);
- LOGGER.info("JACCARD: " + 0.8f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.8f);
+ }
searchModifier.setJaccThresh(0.8f);
runQueries(searchModifier, 50);
- LOGGER.info("JACCARD: " + 0.7f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.7f);
+ }
searchModifier.setJaccThresh(0.7f);
runQueries(searchModifier, 50);
- LOGGER.info("JACCARD: " + 0.6f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.6f);
+ }
searchModifier.setJaccThresh(0.6f);
runQueries(searchModifier, 50);
- LOGGER.info("JACCARD: " + 0.5f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.5f);
+ }
searchModifier.setJaccThresh(0.5f);
runQueries(searchModifier, 50);
}
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index 91fb924..eb2b39c 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -1,3 +1,18 @@
+/*
+ * 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.invertedindex;
import java.io.DataOutputStream;
@@ -5,6 +20,7 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.logging.Level;
import org.junit.Before;
import org.junit.Test;
@@ -14,6 +30,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.PageAllocationException;
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
@@ -113,7 +130,7 @@
}
}
- public void loadData() throws IOException, TreeIndexException {
+ public void loadData() throws IOException, TreeIndexException, PageAllocationException {
List<TokenIdPair> pairs = new ArrayList<TokenIdPair>();
// generate pairs for subsequent sorting and bulk-loading
int id = 0;
@@ -215,7 +232,9 @@
}
// remove trailing newline
strBuilder.deleteCharAt(strBuilder.length() - 1);
- LOGGER.info(strBuilder.toString());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(strBuilder.toString());
+ }
}
}
@@ -239,15 +258,21 @@
public void jaccardQueryTest() throws Exception {
JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
- LOGGER.info("JACCARD: " + 0.9f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.9f);
+ }
searchModifier.setJaccThresh(0.9f);
runQueries(searchModifier, 5);
- LOGGER.info("JACCARD: " + 0.8f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.8f);
+ }
searchModifier.setJaccThresh(0.8f);
runQueries(searchModifier, 5);
- LOGGER.info("JACCARD: " + 0.7f);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("JACCARD: " + 0.7f);
+ }
searchModifier.setJaccThresh(0.7f);
runQueries(searchModifier, 5);
}
@@ -262,15 +287,21 @@
EditDistanceSearchModifier searchModifier = new EditDistanceSearchModifier(
3, 0);
- LOGGER.info("EDIT DISTANCE: " + 1);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("EDIT DISTANCE: " + 1);
+ }
searchModifier.setEdThresh(1);
runQueries(searchModifier, 5);
- LOGGER.info("EDIT DISTANCE: " + 2);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("EDIT DISTANCE: " + 2);
+ }
searchModifier.setEdThresh(2);
runQueries(searchModifier, 5);
- LOGGER.info("EDIT DISTANCE: " + 3);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("EDIT DISTANCE: " + 3);
+ }
searchModifier.setEdThresh(3);
runQueries(searchModifier, 5);
}
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java
index 25c5370..8f619a4 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeTest.java
@@ -18,11 +18,13 @@
import java.io.File;
import java.text.SimpleDateFormat;
import java.util.Date;
+import java.util.logging.Logger;
import org.junit.AfterClass;
public abstract class AbstractRTreeTest {
+ protected static final Logger LOGGER = Logger.getLogger(AbstractRTreeTest.class.getName());
protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat(
"ddMMyy-hhmmssSS");
protected final static String tmpDir = System.getProperty("java.io.tmpdir");
@@ -30,10 +32,6 @@
protected final static String fileName = tmpDir + sep
+ simpleDateFormat.format(new Date());
- protected void print(String str) {
- System.err.print(str);
- }
-
@AfterClass
public static void cleanup() throws Exception {
File f = new File(fileName);
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index ce3bd3c..faf1e88 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -19,6 +19,7 @@
import java.io.File;
import java.nio.ByteBuffer;
import java.util.Random;
+import java.util.logging.Level;
import org.junit.Test;
@@ -42,6 +43,7 @@
import edu.uci.ics.hyracks.dataflow.common.util.TupleUtils;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
@@ -49,7 +51,6 @@
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStats;
import edu.uci.ics.hyracks.storage.am.common.util.TreeIndexStatsGatherer;
@@ -57,7 +58,6 @@
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
import edu.uci.ics.hyracks.storage.am.rtree.util.RTreeUtils;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -155,7 +155,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -199,38 +199,42 @@
tuple.reset(accessor, 0);
- if (i % 1000 == 0) {
- print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x)
+ + " " + Math.min(p1y, p2y) + " "
+ + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y));
+ }
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
}
}
- // rtree.printTree(leafFrame, interiorFrame, recDescSers);
- // System.err.println();
-
String rtreeStats = rtree.printStats();
- print(rtreeStats);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
// disk-order scan
- print("DISK-ORDER SCAN:\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
leafFrame);
- RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
- rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
ITupleReference frameTuple = diskOrderCursor.getTuple();
String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- print(rec + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
}
} catch (Exception e) {
e.printStackTrace();
@@ -243,7 +247,9 @@
TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
interiorFrame, metaFrame);
String string = stats.toString();
- System.err.println(string);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
rtree.close();
bufferCache.closeFile(fileId);
@@ -334,7 +340,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -376,27 +382,27 @@
tuple.reset(accessor, 0);
- if (i % 1000 == 0) {
- print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+ + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+ + " " + Math.max(p1y, p2y));
+ }
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
}
}
- // rtree.printTree(leafFrame, interiorFrame, recDescSers);
- // System.err.println();
-
String rtreeStats = rtree.printStats();
- print(rtreeStats);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
- RTreeOpContext deleteOpCtx = rtree.createOpContext(IndexOp.DELETE);
rnd.setSeed(50);
for (int i = 0; i < 5000; i++) {
@@ -434,16 +440,17 @@
tb.getSize());
tuple.reset(accessor, 0);
-
- if (i % 1000 == 0) {
- print("DELETING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + "\n");
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("DELETING " + i + " " + Math.min(p1x, p2x) + " "
+ + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+ + " " + Math.max(p1y, p2y));
+ }
}
try {
- rtree.delete(tuple, deleteOpCtx);
-
+ indexAccessor.delete(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -455,7 +462,9 @@
TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
interiorFrame, metaFrame);
String string = stats.toString();
- System.err.println(string);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
rtree.close();
bufferCache.closeFile(fileId);
@@ -552,7 +561,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -602,39 +611,43 @@
tuple.reset(accessor, 0);
- if (i % 1000 == 0) {
- print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.min(p1z, p2z) + " "
- + " " + Math.max(p1x, p2x) + " " + Math.max(p1y, p2y)
- + " " + Math.max(p1z, p2z) + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+ + Math.min(p1y, p2y) + " " + Math.min(p1z, p2z)
+ + " " + " " + Math.max(p1x, p2x) + " "
+ + Math.max(p1y, p2y) + " " + Math.max(p1z, p2z));
+ }
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
}
}
- // rtree.printTree(leafFrame, interiorFrame, recDescSers);
- // System.err.println();
-
String rtreeStats = rtree.printStats();
- print(rtreeStats);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
// disk-order scan
- print("DISK-ORDER SCAN:\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
leafFrame);
- RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
- rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
ITupleReference frameTuple = diskOrderCursor.getTuple();
String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- print(rec + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
}
} catch (Exception e) {
e.printStackTrace();
@@ -647,7 +660,9 @@
TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
interiorFrame, metaFrame);
String string = stats.toString();
- System.err.println(string);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
rtree.close();
bufferCache.closeFile(fileId);
@@ -738,7 +753,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -782,38 +797,42 @@
tuple.reset(accessor, 0);
- if (i % 1000 == 0) {
- print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+ + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+ + " " + Math.max(p1y, p2y));
+ }
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
}
}
- // rtree.printTree(leafFrame, interiorFrame, recDescSers);
- // System.err.println();
-
String rtreeStats = rtree.printStats();
- print(rtreeStats);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rtreeStats);
+ }
// disk-order scan
- print("DISK-ORDER SCAN:\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("DISK-ORDER SCAN:");
+ }
TreeDiskOrderScanCursor diskOrderCursor = new TreeDiskOrderScanCursor(
leafFrame);
- RTreeOpContext diskOrderScanOpCtx = rtree.createOpContext(IndexOp.DISKORDERSCAN);
- rtree.diskOrderScan(diskOrderCursor, diskOrderScanOpCtx);
+ indexAccessor.diskOrderScan(diskOrderCursor);
try {
while (diskOrderCursor.hasNext()) {
diskOrderCursor.next();
ITupleReference frameTuple = diskOrderCursor.getTuple();
String rec = TupleUtils.printTuple(frameTuple, recDescSers);
- print(rec + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(rec);
+ }
}
} catch (Exception e) {
e.printStackTrace();
@@ -826,7 +845,9 @@
TreeIndexStats stats = statsGatherer.gatherStats(leafFrame,
interiorFrame, metaFrame);
String string = stats.toString();
- System.err.println(string);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(string);
+ }
rtree.close();
bufferCache.closeFile(fileId);
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index c87f0da..60d0578 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -23,6 +23,7 @@
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Random;
+import java.util.logging.Level;
import org.junit.Test;
@@ -44,6 +45,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
@@ -51,14 +53,12 @@
import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeInteriorFrame;
import edu.uci.ics.hyracks.storage.am.rtree.api.IRTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMInteriorFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreeNSMLeafFrameFactory;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeOpContext;
import edu.uci.ics.hyracks.storage.am.rtree.impls.RTreeSearchCursor;
import edu.uci.ics.hyracks.storage.am.rtree.impls.SearchPredicate;
import edu.uci.ics.hyracks.storage.am.rtree.tuples.RTreeTypeAwareTupleWriterFactory;
@@ -155,7 +155,7 @@
accessor.reset(hyracksFrame);
FrameTupleReference tuple = new FrameTupleReference();
- RTreeOpContext insertOpCtx = rtree.createOpContext(IndexOp.INSERT);
+ ITreeIndexAccessor indexAccessor = rtree.createAccessor();
Random rnd = new Random();
rnd.setSeed(50);
@@ -190,14 +190,16 @@
tuple.reset(accessor, 0);
- if (i % 1000 == 0) {
- print("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ if (i % 1000 == 0) {
+ LOGGER.info("INSERTING " + i + " " + Math.min(p1x, p2x) + " "
+ + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x)
+ + " " + Math.max(p1y, p2y));
+ }
}
try {
- rtree.insert(tuple, insertOpCtx);
+ indexAccessor.insert(tuple);
} catch (TreeIndexException e) {
} catch (Exception e) {
e.printStackTrace();
@@ -234,16 +236,17 @@
tuple.reset(accessor, 0);
- print(i + " Searching for: " + Math.min(p1x, p2x) + " "
- + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
- + Math.max(p1y, p2y) + "\n");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(i + " Searching for: " + Math.min(p1x, p2x) + " "
+ + Math.min(p1y, p2y) + " " + Math.max(p1x, p2x) + " "
+ + Math.max(p1y, p2y));
+ }
ITreeIndexCursor searchCursor = new RTreeSearchCursor(
interiorFrame, leafFrame);
SearchPredicate searchPredicate = new SearchPredicate(tuple, cmp);
- RTreeOpContext searchOpCtx = rtree.createOpContext(IndexOp.SEARCH);
- rtree.search(searchCursor, searchPredicate, searchOpCtx);
+ indexAccessor.search(searchCursor, searchPredicate);
ArrayList<Integer> results = new ArrayList<Integer>();
try {
@@ -264,9 +267,10 @@
} finally {
searchCursor.close();
}
-
- System.err.println("There are " + results.size()
- + " objects that satisfy the query");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("There are " + results.size()
+ + " objects that satisfy the query");
+ }
}
rtree.close();
diff --git a/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java
new file mode 100644
index 0000000..6c91cdd
--- /dev/null
+++ b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTests.java
@@ -0,0 +1,180 @@
+package edu.uci.ics.hyracks.storage.common;
+
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IIOManager.FileReadWriteMode;
+import edu.uci.ics.hyracks.api.io.IIOManager.FileSyncMode;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public class BufferCacheRegressionTests {
+ protected static final String tmpDir = System.getProperty("java.io.tmpdir");
+ protected static final String sep = System.getProperty("file.separator");
+
+ protected String fileName = tmpDir + sep + "flushTestFile";
+
+ private static final int PAGE_SIZE = 256;
+ private static final int HYRACKS_FRAME_SIZE = PAGE_SIZE;
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
+ // We want to test the following behavior when reclaiming a file slot in the
+ // buffer cache:
+ // 1. If the file being evicted was deleted, then its dirty pages should be
+ // invalidated, but most not be flushed.
+ // 2. If the file was not deleted, then we must flush its dirty pages.
+ @Test
+ public void testFlushBehaviorOnFileEviction() throws IOException {
+ File f = new File(fileName);
+ if (f.exists()) {
+ f.delete();
+ }
+ flushBehaviorTest(true);
+ flushBehaviorTest(false);
+ }
+
+ private void flushBehaviorTest(boolean deleteFile) throws IOException {
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, 10, 1);
+
+ IBufferCache bufferCache = TestStorageManagerComponentHolder
+ .getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder
+ .getFileMapProvider(ctx);
+
+ FileReference firstFileRef = new FileReference(new File(fileName));
+ bufferCache.createFile(firstFileRef);
+ int firstFileId = fmp.lookupFileId(firstFileRef);
+ bufferCache.openFile(firstFileId);
+
+ // Fill the first page with known data and make it dirty by write
+ // latching it.
+ ICachedPage writePage = bufferCache.pin(
+ BufferedFileHandle.getDiskPageId(firstFileId, 0), true);
+ writePage.acquireWriteLatch();
+ try {
+ ByteBuffer buf = writePage.getBuffer();
+ for (int i = 0; i < buf.capacity(); i++) {
+ buf.put(Byte.MAX_VALUE);
+ }
+ } finally {
+ writePage.releaseWriteLatch();
+ bufferCache.unpin(writePage);
+ }
+ bufferCache.closeFile(firstFileId);
+ if (deleteFile) {
+ bufferCache.deleteFile(firstFileId);
+ }
+
+ // Create a file with the same name.
+ FileReference secondFileRef = new FileReference(new File(fileName));
+ bufferCache.createFile(secondFileRef);
+ int secondFileId = fmp.lookupFileId(secondFileRef);
+
+ // This open will replace the firstFileRef's slot in the BufferCache,
+ // causing it's pages to be cleaned up. We want to make sure that those
+ // dirty pages are not flushed to the disk, because the file was
+ // declared as deleted, and
+ // somebody might be already using the same filename again (having been
+ // assigned a different fileId).
+ bufferCache.openFile(secondFileId);
+
+ // Manually open the file and inspect it's contents. We cannot simply
+ // ask the BufferCache to pin the page, because it would return the same
+ // physical memory again, and for performance reasons pages are never
+ // reset with 0's.
+ IIOManager ioManager = ctx.getIOManager();
+ FileReference testFileRef = new FileReference(new File(fileName));
+ FileHandle testFileHandle = new FileHandle(testFileRef);
+ testFileHandle.open(FileReadWriteMode.READ_ONLY,
+ FileSyncMode.METADATA_SYNC_DATA_SYNC);
+ ByteBuffer testBuffer = ByteBuffer.allocate(PAGE_SIZE);
+ ioManager.syncRead(testFileHandle, 0, testBuffer);
+ for (int i = 0; i < testBuffer.capacity(); i++) {
+ if (deleteFile) {
+ // We deleted the file. We expect to see a clean buffer.
+ if (testBuffer.get(i) == Byte.MAX_VALUE) {
+ fail("Page 0 of deleted file was fazily flushed in openFile(), "
+ + "corrupting the data of a newly created file with the same name.");
+ }
+ } else {
+ // We didn't delete the file. We expect to see a buffer full of
+ // Byte.MAX_VALUE.
+ if (testBuffer.get(i) != Byte.MAX_VALUE) {
+ fail("Page 0 of closed file was not flushed when properly, when reclaiming the file slot of fileId 0 in the BufferCache.");
+ }
+ }
+ }
+ testFileHandle.close();
+ bufferCache.closeFile(secondFileId);
+ if (deleteFile) {
+ bufferCache.deleteFile(secondFileId);
+ }
+ bufferCache.close();
+ }
+
+ // Tests the behavior of the BufferCache when more than all pages are
+ // pinned. We expect an exception.
+ @Test
+ public void testPinningAllPages() throws HyracksDataException {
+ int numPages = 10;
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, numPages, 1);
+
+ IBufferCache bufferCache = TestStorageManagerComponentHolder
+ .getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder
+ .getFileMapProvider(ctx);
+
+ FileReference firstFileRef = new FileReference(new File(fileName));
+ bufferCache.createFile(firstFileRef);
+ int fileId = fmp.lookupFileId(firstFileRef);
+ bufferCache.openFile(fileId);
+
+ // Pin all pages.
+ ICachedPage[] pages = new ICachedPage[numPages];
+ for (int i = 0; i < numPages; ++i) {
+ pages[i] = bufferCache.pin(
+ BufferedFileHandle.getDiskPageId(fileId, i), true);
+ }
+
+ // Try to pin another page. We expect a HyracksDataException.
+ ICachedPage errorPage = null;
+ try {
+ errorPage = bufferCache.pin(
+ BufferedFileHandle.getDiskPageId(fileId, numPages), true);
+ } catch (HyracksDataException e) {
+ // This is the expected outcome.
+ // The BufferCache should still be able to function properly.
+ // Try unpinning all pages.
+ for (int i = 0; i < numPages; ++i) {
+ bufferCache.unpin(pages[i]);
+ }
+ // Now try pinning the page that failed above again.
+ errorPage = bufferCache.pin(
+ BufferedFileHandle.getDiskPageId(fileId, numPages), true);
+ // Unpin it.
+ bufferCache.unpin(errorPage);
+ // Cleanup.
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
+ return;
+ } catch (Exception e) {
+ fail("Expected a HyracksDataException when pinning more pages than available but got another exception: "
+ + e.getMessage());
+ }
+ fail("Expected a HyracksDataException when pinning more pages than available.");
+ }
+}