Polished inverted index. Added integration test for inverted index bulk load.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_btree_updates_next@775 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
similarity index 96%
rename from hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
rename to hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
index c84ef74..fd558be 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
@@ -28,7 +28,7 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
+public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
@Test
public void tokenizerTest() throws Exception {
@@ -52,9 +52,9 @@
IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
tokenFactory);
int[] tokenFields = { 1 };
- int[] projFields = { 0 };
+ int[] keyFields = { 0 };
BinaryTokenizerOperatorDescriptor binaryTokenizer = new BinaryTokenizerOperatorDescriptor(spec,
- tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
+ tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
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..26ac7dc
--- /dev/null
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -0,0 +1,282 @@
+/*
+ * 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.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
+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.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.btree.impls.BTree;
+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.TreeIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+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.impls.InvertedIndex;
+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.TestInvertedIndexRegistryProvider;
+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 WordInvertedIndexTest extends AbstractIntegrationTest {
+ static {
+ TestStorageManagerComponentHolder.init(8192, 20, 20);
+ }
+
+ private IStorageManagerInterface storageManager = new TestStorageManagerInterface();
+ private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider = new TestTreeIndexRegistryProvider();
+ private IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider = new TestInvertedIndexRegistryProvider();
+ private ITreeIndexOpHelperFactory btreeOpHelperFactory = new BTreeOpHelperFactory();
+
+ 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 int btreeFieldCount = 5;
+ private ITypeTrait[] btreeTypeTraits = new ITypeTrait[btreeFieldCount];
+ private int btreeKeyFieldCount = 1;
+ private IBinaryComparatorFactory[] btreeComparatorFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
+ private TypeAwareTupleWriterFactory btreeTupleWriterFactory = new TypeAwareTupleWriterFactory(btreeTypeTraits);
+ private ITreeIndexFrameFactory btreeInteriorFrameFactory = new BTreeNSMInteriorFrameFactory(btreeTupleWriterFactory);
+ private ITreeIndexFrameFactory btreeLeafFrameFactory = new BTreeNSMLeafFrameFactory(btreeTupleWriterFactory);
+ private RecordDescriptor btreeRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE });
+
+ // 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 });
+
+ @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 dictionary BTree.
+ btreeTypeTraits[0] = ITypeTrait.VARLEN_TYPE_TRAIT;
+ btreeTypeTraits[1] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ btreeTypeTraits[2] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ btreeTypeTraits[3] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ btreeTypeTraits[4] = ITypeTrait.INTEGER_TYPE_TRAIT;
+ btreeComparatorFactories[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 test() {
+ // TODO: perform inverted index searches.
+ }
+
+ 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, treeIndexRegistryProvider, primaryFileSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
+ btreeOpHelperFactory);
+ 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, treeIndexRegistryProvider, primaryFileSplitProvider, primaryInteriorFrameFactory,
+ primaryLeafFrameFactory, primaryTypeTraits, primaryComparatorFactories, true, lowKeyFields,
+ highKeyFields, true, true, btreeOpHelperFactory);
+ 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);
+ PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+ 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) {
+ ITokenFactory tokenFactory = new UTF8WordTokenFactory();
+ IBinaryTokenizerFactory tokenizerFactory = new DelimitedUTF8StringBinaryTokenizerFactory(true, false,
+ tokenFactory);
+ 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,
+ storageManager, fieldPermutation, btreeFileSplitProvider, treeIndexRegistryProvider,
+ btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories,
+ BTree.DEFAULT_FILL_FACTOR, btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider,
+ invListsTypeTraits, invListsComparatorFactories);
+ 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);
+ }
+}
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 919ba18..48ee8d4 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;
@@ -32,6 +31,7 @@
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.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;
@@ -98,33 +98,9 @@
treeIndexOpHelper.init();
btree = (BTree) treeIndexOpHelper.getTreeIndex();
- // 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);
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/dataflow/TreeIndexOpHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
index 6fea7e5..353b013 100644
--- 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
@@ -24,6 +24,7 @@
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.am.common.util.IndexUtils;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -74,7 +75,6 @@
// 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();
@@ -83,20 +83,11 @@
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);
+ }
+ cmp = IndexUtils.createMultiComparator(opDesc.getTreeIndexComparatorFactories());
treeIndex = createTreeIndex();
if (mode == IndexHelperOpenMode.CREATE) {
- try {
- treeIndex.create(indexFileId);
- } catch (Exception e) {
- e.printStackTrace();
- throw new HyracksDataException(e);
- }
+ treeIndex.create(indexFileId);
}
treeIndex.open(indexFileId);
treeIndexRegistry.register(indexFileId, treeIndex);
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/IInvertedIndexOperatorDescriptorHelper.java
index b3afe4a..5085e31 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/IInvertedIndexOperatorDescriptorHelper.java
@@ -23,11 +23,11 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
public interface IInvertedIndexOperatorDescriptorHelper extends ITreeIndexOperatorDescriptorHelper {
- public IFileSplitProvider getInvIndexFileSplitProvider();
+ public IFileSplitProvider getInvListsFileSplitProvider();
- public IBinaryComparatorFactory[] getInvIndexComparatorFactories();
+ public IBinaryComparatorFactory[] getInvListsComparatorFactories();
- public ITypeTrait[] getInvIndexTypeTraits();
+ public ITypeTrait[] getInvListsTypeTraits();
public IIndexRegistryProvider<InvertedIndex> getInvIndexRegistryProvider();
}
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..4dc655e 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,7 +21,6 @@
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.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
@@ -41,26 +40,26 @@
// btree
protected final IFileSplitProvider btreeFileSplitProvider;
protected final IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
- protected final ITreeIndexFrameFactory interiorFrameFactory;
- protected final ITreeIndexFrameFactory leafFrameFactory;
+ protected final ITreeIndexFrameFactory btreeInteriorFrameFactory;
+ protected final ITreeIndexFrameFactory btreeLeafFrameFactory;
protected final ITypeTrait[] btreeTypeTraits;
protected final IBinaryComparatorFactory[] btreeComparatorFactories;
- protected final ITreeIndexOpHelperFactory opHelperFactory;
+ protected final ITreeIndexOpHelperFactory btreeOpHelperFactory;
// inverted index
- protected final IFileSplitProvider invIndexFileSplitProvider;
+ protected final IFileSplitProvider invListsFileSplitProvider;
protected final IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider;
- protected final ITypeTrait[] invIndexTypeTraits;
- protected final IBinaryComparatorFactory[] invIndexComparatorFactories;
+ protected final ITypeTrait[] invListsTypeTraits;
+ protected final IBinaryComparatorFactory[] invListComparatorFactories;
public AbstractInvertedIndexOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity,
RecordDescriptor recDesc, IStorageManagerInterface storageManager,
IFileSplitProvider btreeFileSplitProvider, IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider,
- ITreeIndexFrameFactory interiorFrameFactory, ITreeIndexFrameFactory leafFrameFactory,
+ ITreeIndexFrameFactory btreeInteriorFrameFactory, ITreeIndexFrameFactory btreeLeafFrameFactory,
ITypeTrait[] btreeTypeTraits, IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
- ITreeIndexOpHelperFactory opHelperFactory, IFileSplitProvider invIndexFileSplitProvider,
- IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
- IBinaryComparatorFactory[] invIndexComparatorFactories) {
+ ITreeIndexOpHelperFactory btreeOpHelperFactory, IFileSplitProvider invListsFileSplitProvider,
+ IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
+ IBinaryComparatorFactory[] invListComparatorFactories) {
super(spec, inputArity, outputArity);
// general
@@ -69,20 +68,21 @@
// btree
this.btreeFileSplitProvider = btreeFileSplitProvider;
this.treeIndexRegistryProvider = treeIndexRegistryProvider;
- this.interiorFrameFactory = interiorFrameFactory;
- this.leafFrameFactory = leafFrameFactory;
+ this.btreeInteriorFrameFactory = btreeInteriorFrameFactory;
+ this.btreeLeafFrameFactory = btreeLeafFrameFactory;
this.btreeTypeTraits = btreeTypeTraits;
this.btreeComparatorFactories = btreeComparatorFactories;
- this.opHelperFactory = opHelperFactory;
+ this.btreeOpHelperFactory = btreeOpHelperFactory;
// inverted index
- this.invIndexFileSplitProvider = invIndexFileSplitProvider;
+ this.invListsFileSplitProvider = invListsFileSplitProvider;
this.invIndexRegistryProvider = invIndexRegistryProvider;
- this.invIndexTypeTraits = invIndexTypeTraits;
- this.invIndexComparatorFactories = invIndexComparatorFactories;
+ this.invListsTypeTraits = invListsTypeTraits;
+ this.invListComparatorFactories = invListComparatorFactories;
- if (outputArity > 0)
+ if (outputArity > 0) {
recordDescriptors[0] = recDesc;
+ }
}
@Override
@@ -107,12 +107,12 @@
@Override
public ITreeIndexFrameFactory getTreeIndexInteriorFactory() {
- return interiorFrameFactory;
+ return btreeInteriorFrameFactory;
}
@Override
public ITreeIndexFrameFactory getTreeIndexLeafFactory() {
- return leafFrameFactory;
+ return btreeLeafFrameFactory;
}
@Override
@@ -136,22 +136,22 @@
}
@Override
- public IBinaryComparatorFactory[] getInvIndexComparatorFactories() {
- return invIndexComparatorFactories;
+ public IBinaryComparatorFactory[] getInvListsComparatorFactories() {
+ return invListComparatorFactories;
}
@Override
- public IFileSplitProvider getInvIndexFileSplitProvider() {
- return invIndexFileSplitProvider;
+ public IFileSplitProvider getInvListsFileSplitProvider() {
+ return invListsFileSplitProvider;
}
@Override
- public ITypeTrait[] getInvIndexTypeTraits() {
- return invIndexTypeTraits;
+ public ITypeTrait[] getInvListsTypeTraits() {
+ return invListsTypeTraits;
}
@Override
public ITreeIndexOpHelperFactory getTreeIndexOpHelperFactory() {
- return opHelperFactory;
+ return btreeOpHelperFactory;
}
}
\ 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 b5b1393..1041deb 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
@@ -32,17 +32,17 @@
private final IBinaryTokenizerFactory tokenizerFactory;
// 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;
}
@@ -50,6 +50,6 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
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 1ab72fe..a250750 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
@@ -27,7 +27,6 @@
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.invertedindex.api.IInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -37,29 +36,27 @@
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,
+ IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, ITreeIndexFrameFactory btreeInteriorFrameFactory,
+ ITreeIndexFrameFactory btreeLeafFrameFactory, ITypeTrait[] btreeTypeTraits,
IBinaryComparatorFactory[] btreeComparatorFactories, float btreeFillFactor,
- ITreeIndexOpHelperFactory opHelperFactory, IFileSplitProvider invIndexFileSplitProvider,
- IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invIndexTypeTraits,
- IBinaryComparatorFactory[] invIndexComparatorFactories, IInvertedListBuilder invListBuilder) {
+ ITreeIndexOpHelperFactory btreeOpHelperFactory, IFileSplitProvider invListsFileSplitProvider,
+ IIndexRegistryProvider<InvertedIndex> invIndexRegistryProvider, ITypeTrait[] invListsTypeTraits,
+ IBinaryComparatorFactory[] invListComparatorFactories) {
super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, treeIndexRegistryProvider,
- interiorFrameFactory, leafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeFillFactor,
- opHelperFactory, invIndexFileSplitProvider, invIndexRegistryProvider, invIndexTypeTraits,
- invIndexComparatorFactories);
+ btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeTypeTraits, btreeComparatorFactories, btreeFillFactor,
+ btreeOpHelperFactory, invListsFileSplitProvider, invIndexRegistryProvider, invListsTypeTraits,
+ invListComparatorFactories);
this.fieldPermutation = fieldPermutation;
this.btreeFillFactor = btreeFillFactor;
- this.invListBuilder = invListBuilder;
}
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, btreeFillFactor,
- invListBuilder, recordDescProvider);
+ 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 988f073..7df754a 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
@@ -27,10 +27,11 @@
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.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 final TreeIndexOpHelper btreeOpHelper;
private float btreeFillFactor;
private final InvertedIndexOpHelper invIndexOpHelper;
@@ -46,44 +47,52 @@
public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
- IInvertedListBuilder invListBuilder, IRecordDescriptorProvider recordDescProvider) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
+ IRecordDescriptorProvider recordDescProvider) {
+ btreeOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.CREATE);
- invIndexOpHelper = new InvertedIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.CREATE);
+ invIndexOpHelper = new InvertedIndexOpHelper(btreeOpHelper, opDesc, ctx, partition);
this.btreeFillFactor = btreeFillFactor;
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) btreeOpHelper
.getOperatorDescriptor();
RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
- // btree
+ // BTree.
try {
- treeIndexOpHelper.init();
- treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
+ btreeOpHelper.init();
+ btreeOpHelper.getTreeIndex().open(btreeOpHelper.getIndexFileId());
} catch (Exception e) {
- // cleanup in case of failure
- treeIndexOpHelper.deinit();
- throw new HyracksDataException(e);
+ // Cleanup in case of failure/
+ btreeOpHelper.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);
} catch (Exception e) {
- // cleanup in case of failure
+ // Cleanup in case of failure.
invIndexOpHelper.deinit();
- throw new HyracksDataException(e);
+ if (e instanceof HyracksDataException) {
+ throw (HyracksDataException) e;
+ } else {
+ throw new HyracksDataException(e);
+ }
}
}
@@ -94,10 +103,10 @@
for (int i = 0; i < tupleCount; i++) {
tuple.reset(accessor, i);
try {
- invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
- } catch (PageAllocationException e) {
- throw new HyracksDataException(e);
- }
+ invIndexOpHelper.getInvIndex().bulkLoadAddTuple(bulkLoadCtx, tuple);
+ } catch (PageAllocationException e) {
+ throw new HyracksDataException(e);
+ }
}
}
@@ -106,9 +115,9 @@
try {
invIndexOpHelper.getInvIndex().endBulkLoad(bulkLoadCtx);
} catch (PageAllocationException e) {
- throw new HyracksDataException(e);
- } finally {
- treeIndexOpHelper.deinit();
+ throw new HyracksDataException(e);
+ } finally {
+ btreeOpHelper.deinit();
}
}
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
index 684bb20..33b0eef 100644
--- 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
@@ -15,16 +15,15 @@
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.dataflow.TreeIndexOpHelper;
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.IInvertedIndexOperatorDescriptorHelper;
import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -32,6 +31,8 @@
public final class InvertedIndexOpHelper {
+ private final TreeIndexOpHelper btreeOpHelper;
+
private InvertedIndex invIndex;
private int invIndexFileId = -1;
private int partition;
@@ -39,99 +40,63 @@
private IInvertedIndexOperatorDescriptorHelper opDesc;
private IHyracksTaskContext ctx;
- private IndexHelperOpenMode mode;
+ public InvertedIndexOpHelper(TreeIndexOpHelper btreeOpHelper,
+ IInvertedIndexOperatorDescriptorHelper opDesc,
+ final IHyracksTaskContext ctx, int partition) {
+ this.btreeOpHelper = btreeOpHelper;
+ this.opDesc = opDesc;
+ this.ctx = ctx;
+ this.partition = partition;
+ }
- public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
- int partition, IndexHelperOpenMode mode) {
- this.opDesc = opDesc;
- this.ctx = ctx;
- this.mode = mode;
- this.partition = partition;
- }
-
+ // TODO: This is very similar to TreeIndexOpHelper. Maybe we can somehow merge them?
public void init() throws HyracksDataException {
IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
- IFileSplitProvider fileSplitProvider = opDesc.getInvIndexFileSplitProvider();
+ IFileSplitProvider fileSplitProvider = opDesc.getInvListsFileSplitProvider();
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;
-
+ 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
+ // 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();
- }
- }
- }
+ // 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);
+ // 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) {
+ // Create and register the inverted index.
+ MultiComparator cmp = IndexUtils.createMultiComparator(opDesc
+ .getInvListsComparatorFactories());
+ // Assumes btreeOpHelper.init() has already been called.
+ BTree btree = (BTree) btreeOpHelper.getTreeIndex();
+ invIndex = new InvertedIndex(bufferCache, btree,
+ opDesc.getInvListsTypeTraits(), cmp);
+ invIndex.open(invIndexFileId);
+ invIndexRegistry.register(invIndexFileId, invIndex);
+ }
+ } finally {
+ invIndexRegistry.unlock();
+ }
+ }
public void deinit() throws HyracksDataException {
if (invIndexFileId != -1) {
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..db3ad66
--- /dev/null
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/util/InvertedIndexUtils.java
@@ -0,0 +1,21 @@
+/*
+ * 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;
+
+
+public class InvertedIndexUtils {
+
+}
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/TestInvertedIndexRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestInvertedIndexRegistryProvider.java
new file mode 100644
index 0000000..c7ee98e
--- /dev/null
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestInvertedIndexRegistryProvider.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.test.support;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
+
+public class TestInvertedIndexRegistryProvider implements IIndexRegistryProvider<InvertedIndex> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IndexRegistry<InvertedIndex> getRegistry(IHyracksTaskContext ctx) {
+ return TestStorageManagerComponentHolder.getInvertedIndexRegistry(ctx);
+ }
+}
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..a0aa44b 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
@@ -17,6 +17,7 @@
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.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.invertedindex.impls.InvertedIndex;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -31,6 +32,7 @@
private static IBufferCache bufferCache;
private static IFileMapProvider fileMapProvider;
private static IndexRegistry<ITreeIndex> treeIndexRegistry;
+ private static IndexRegistry<InvertedIndex> invIndexRegistry;
private static int pageSize;
private static int numPages;
@@ -43,6 +45,7 @@
bufferCache = null;
fileMapProvider = null;
treeIndexRegistry = null;
+ invIndexRegistry = null;
}
public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
@@ -69,4 +72,11 @@
}
return treeIndexRegistry;
}
+
+ public synchronized static IndexRegistry<InvertedIndex> getInvertedIndexRegistry(IHyracksTaskContext ctx) {
+ if (invIndexRegistry == null) {
+ invIndexRegistry = new IndexRegistry<InvertedIndex>();
+ }
+ return invIndexRegistry;
+ }
}
\ 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/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 0b4bd62..f9749cf 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/BTreeStatsTest.java
@@ -28,7 +28,6 @@
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.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;
@@ -39,6 +38,7 @@
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;
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;
@@ -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);
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
index bb15de9..e5ed5c2 100644
--- 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
@@ -56,7 +56,6 @@
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.ITreeIndexAccessor;
@@ -71,6 +70,7 @@
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.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.common.util.IndexUtils;
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;
@@ -115,7 +115,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);
@@ -342,7 +342,7 @@
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -545,7 +545,7 @@
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -730,7 +730,7 @@
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -948,7 +948,7 @@
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = UTF8StringBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -1120,7 +1120,7 @@
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
@@ -1283,7 +1283,7 @@
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
cmpFactories[1] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
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 7834419..79134ae 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
@@ -47,7 +47,6 @@
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;
@@ -58,6 +57,7 @@
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 RangeSearchCursorTest extends AbstractBTreeTest {
// Declare fields
@@ -89,7 +89,7 @@
IBinaryComparatorFactory[] cmpFactories = new IBinaryComparatorFactory[keyFieldCount];
cmpFactories[0] = IntegerBinaryComparatorFactory.INSTANCE;
- MultiComparator cmp = BTreeUtils.createMultiComparator(cmpFactories);
+ MultiComparator cmp = IndexUtils.createMultiComparator(cmpFactories);
ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
tupleWriterFactory);
@@ -177,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);
@@ -262,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);
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..0f3d944 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;
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 5847520..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;