Carry filter in 2ndary-to-primary index search
Change-Id: I287f1dbd230aa649f1350114abf0a1d47e2bb53c
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1720
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
BAD: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Luo Chen <cluo8@uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 3c2bee1..cdb9b5b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -211,7 +211,7 @@
filterFields, storageComponentProvider.getMetadataPageManagerFactory());
BTreeSearchOperatorNodePushable searchOp = new BTreeSearchOperatorNodePushable(searchOpDesc, ctx, 0,
primaryIndexInfo.getSearchRecordDescriptorProvider(), /*primaryIndexInfo.primaryKeyIndexes*/null,
- /*primaryIndexInfo.primaryKeyIndexes*/null, true, true, filterFields, filterFields);
+ /*primaryIndexInfo.primaryKeyIndexes*/null, true, true, false, filterFields, filterFields);
emptyTupleOp.setFrameWriter(0, searchOp,
primaryIndexInfo.getSearchRecordDescriptorProvider().getInputRecordDescriptor(null, 0));
searchOp.setOutputFrameWriter(0, countOp, primaryIndexInfo.rDesc);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 69a2020..6e3b5d7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -40,7 +40,7 @@
IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, int[] lowKeyFields,
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) throws HyracksDataException {
super(opDesc, ctx, partition, recordDescProvider, lowKeyFields, highKeyFields, lowKeyInclusive,
- highKeyInclusive, null, null);
+ highKeyInclusive, false, null, null);
}
// We override the open function to search a specific version of the index
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 237e2c0..29c7242 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -39,7 +39,7 @@
public ExternalRTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) throws HyracksDataException {
- super(opDesc, ctx, partition, recordDescProvider, keyFields, null, null);
+ super(opDesc, ctx, partition, recordDescProvider, keyFields, false, null, null);
}
// We override this method to specify the searched version of the index
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index 94252b3..8431640 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -19,13 +19,8 @@
package org.apache.hyracks.tests.am.btree;
-import java.io.DataOutput;
-import java.io.File;
-
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileSplit;
@@ -35,8 +30,6 @@
import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -53,19 +46,42 @@
import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.common.IStorageManager;
import org.apache.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
import org.apache.hyracks.test.support.TestIndexLifecycleManagerProvider;
-import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.test.support.TestStorageManager;
+import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
import org.apache.hyracks.tests.integration.AbstractIntegrationTest;
import org.junit.After;
import org.junit.Before;
+import java.io.DataOutput;
+import java.io.File;
+
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.inputParserFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.inputRecordDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBtreeFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryFieldPermutation;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryFilterFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryKeyFieldCount;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryRecDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryTypeTraits;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryBtreeFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryFieldPermutationA;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryFieldPermutationB;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryFilterFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryRecDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryTypeTraits;
+
public abstract class AbstractBTreeOperatorTest extends AbstractIntegrationTest {
static {
TestStorageManagerComponentHolder.init(8192, 20, 20);
@@ -73,38 +89,13 @@
protected final IStorageManager storageManager = new TestStorageManager();
protected final IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
- protected IIndexDataflowHelperFactory dataflowHelperFactory;
-
- // field, type and key declarations for primary index
- protected final int primaryFieldCount = 6;
- protected final ITypeTraits[] primaryTypeTraits = new ITypeTraits[primaryFieldCount];
- protected final int primaryKeyFieldCount = 1;
- protected final IBinaryComparatorFactory[] primaryComparatorFactories =
- new IBinaryComparatorFactory[primaryKeyFieldCount];
- protected final int[] primaryBloomFilterKeyFields = new int[primaryKeyFieldCount];
-
- protected final RecordDescriptor primaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
+ protected IIndexDataflowHelperFactory primaryDataflowHelperFactory;
+ protected IIndexDataflowHelperFactory secondaryDataflowHelperFactory;
// to be set by subclasses
- protected String primaryFileName;
protected IFileSplitProvider primarySplitProvider;
- protected IPageManagerFactory pageManagerFactory = new LinkedMetadataPageManagerFactory();
+ protected IPageManagerFactory pageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
- // field, type and key declarations for secondary indexes
- protected final int secondaryFieldCount = 2;
- protected final ITypeTraits[] secondaryTypeTraits = new ITypeTraits[secondaryFieldCount];
- protected final int secondaryKeyFieldCount = 2;
- protected final IBinaryComparatorFactory[] secondaryComparatorFactories =
- new IBinaryComparatorFactory[secondaryKeyFieldCount];
- protected final int[] secondaryBloomFilterKeyFields = new int[secondaryKeyFieldCount];
-
- protected final RecordDescriptor secondaryRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
-
- protected String secondaryFileName;
protected IFileSplitProvider secondarySplitProvider;
protected ITreeIndexOperatorTestHelper testHelper;
@@ -116,44 +107,27 @@
@Before
public void setup() throws Exception {
testHelper = createTestHelper();
- dataflowHelperFactory = createDataFlowHelperFactory();
- primaryFileName = testHelper.getPrimaryIndexName();
- primarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID,
- primaryFileName) });
- secondaryFileName = testHelper.getSecondaryIndexName();
- secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID,
- secondaryFileName) });
-
-
- // field, type and key declarations for primary index
- primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
- primaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
- primaryTypeTraits[2] = UTF8StringPointable.TYPE_TRAITS;
- primaryTypeTraits[3] = UTF8StringPointable.TYPE_TRAITS;
- primaryTypeTraits[4] = UTF8StringPointable.TYPE_TRAITS;
- primaryTypeTraits[5] = UTF8StringPointable.TYPE_TRAITS;
- primaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
- primaryBloomFilterKeyFields[0] = 0;
-
- // field, type and key declarations for secondary indexes
- secondaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
- secondaryTypeTraits[1] = UTF8StringPointable.TYPE_TRAITS;
- secondaryComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
- secondaryComparatorFactories[1] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
- secondaryBloomFilterKeyFields[0] = 0;
- secondaryBloomFilterKeyFields[1] = 1;
+ primaryDataflowHelperFactory = createDataFlowHelperFactory(primaryBtreeFields, primaryFilterFields);
+ secondaryDataflowHelperFactory = createDataFlowHelperFactory(secondaryBtreeFields, secondaryFilterFields);
+ String primaryFileName = testHelper.getPrimaryIndexName();
+ primarySplitProvider =
+ new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID, primaryFileName) });
+ String secondaryFileName = testHelper.getSecondaryIndexName();
+ secondarySplitProvider =
+ new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID, secondaryFileName) });
}
- protected abstract IIndexDataflowHelperFactory createDataFlowHelperFactory();
+ protected abstract IIndexDataflowHelperFactory createDataFlowHelperFactory(int [] btreeFields, int[] filterFields);
public void createPrimaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
TransientLocalResourceFactoryProvider localResourceFactoryProvider =
new TransientLocalResourceFactoryProvider();
- TreeIndexCreateOperatorDescriptor primaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
- lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- primaryBloomFilterKeyFields, dataflowHelperFactory, localResourceFactoryProvider,
- NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
+ TreeIndexCreateOperatorDescriptor primaryCreateOp =
+ new TreeIndexCreateOperatorDescriptor(spec, storageManager, lcManagerProvider, primarySplitProvider,
+ primaryTypeTraits, primaryComparatorFactories, primaryBloomFilterKeyFields,
+ primaryDataflowHelperFactory, localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryCreateOp, NC1_ID);
spec.addRoot(primaryCreateOp);
runTest(spec);
@@ -165,19 +139,10 @@
FileSplit[] ordersSplits = new FileSplit[] { new ManagedFileSplit(NC1_ID,
"data" + File.separator + "tpch0.001" + File.separator + "orders-part1.tbl") };
IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer() });
+ RecordDescriptor ordersDesc = inputRecordDesc;
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);
+ new DelimitedDataTupleParserFactory(inputParserFactories, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 0 },
@@ -185,11 +150,11 @@
ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
- int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
- TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- primaryRecDesc, storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, primaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true,
- dataflowHelperFactory, pageManagerFactory);
+ TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad =
+ new TreeIndexBulkLoadOperatorDescriptor(spec, primaryRecDesc, storageManager, lcManagerProvider,
+ primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ primaryBloomFilterKeyFields, primaryFieldPermutation, 0.7f, true, 1000L, true,
+ primaryDataflowHelperFactory, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
NullSinkOperatorDescriptor nsOpDesc = new NullSinkOperatorDescriptor(spec);
@@ -207,10 +172,11 @@
JobSpecification spec = new JobSpecification();
TransientLocalResourceFactoryProvider localResourceFactoryProvider =
new TransientLocalResourceFactoryProvider();
- TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
- storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, secondaryBloomFilterKeyFields, dataflowHelperFactory,
- localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
+ TreeIndexCreateOperatorDescriptor secondaryCreateOp =
+ new TreeIndexCreateOperatorDescriptor(spec, storageManager, lcManagerProvider, secondarySplitProvider,
+ secondaryTypeTraits, secondaryComparatorFactories, secondaryBloomFilterKeyFields,
+ secondaryDataflowHelperFactory, localResourceFactoryProvider,
+ NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
spec.addRoot(secondaryCreateOp);
runTest(spec);
@@ -227,36 +193,40 @@
new UTF8StringSerializerDeserializer().serialize("0", dos);
tb.addFieldEndOffset();
- ISerializerDeserializer[] keyRecDescSers = { new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer() };
- RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+ RecordDescriptor keyRecDesc = secondaryRecDesc;
- ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
- keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
+ 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
// scan primary index
- BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
- false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, new LinkedMetadataPageManagerFactory());
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, primaryRecDesc, storageManager, lcManagerProvider,
+ primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
+ secondaryDataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null,
+ null, new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
// sort based on secondary keys
- ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, 1000, new int[] { 3, 0 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, primaryRecDesc);
+ ExternalSortOperatorDescriptor sorter =
+ new ExternalSortOperatorDescriptor(spec, 1000, secondaryFieldPermutationA,
+ new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, primaryRecDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID);
// load secondary index
- int[] fieldPermutation = { 3, 0 };
- TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true,
- dataflowHelperFactory, pageManagerFactory);
+ int[] fieldPermutation = secondaryFieldPermutationA;
+ TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad =
+ new TreeIndexBulkLoadOperatorDescriptor(spec, secondaryRecDesc, storageManager, lcManagerProvider,
+ secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
+ secondaryBloomFilterKeyFields, fieldPermutation, 0.7f, true, 1000L, true,
+ secondaryDataflowHelperFactory, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
NullSinkOperatorDescriptor nsOpDesc = new NullSinkOperatorDescriptor(spec);
@@ -278,37 +248,28 @@
FileSplit[] ordersSplits = new FileSplit[] { new ManagedFileSplit(NC1_ID,
"data" + File.separator + "tpch0.002" + File.separator + "orders-part2.tbl") };
IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
- new UTF8StringSerializerDeserializer() });
+ RecordDescriptor ordersDesc = inputRecordDesc;
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);
+ new DelimitedDataTupleParserFactory(inputParserFactories, '|'), 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, lcManagerProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, primaryBloomFilterKeyFields, primaryFieldPermutation, pipelineOperation,
- dataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
+ new TreeIndexInsertUpdateDeleteOperatorDescriptor(spec, ordersDesc, storageManager, lcManagerProvider,
+ primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ primaryBloomFilterKeyFields, primaryFieldPermutation, pipelineOperation,
+ primaryDataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeInsertOp, NC1_ID);
// first secondary index
- int[] fieldPermutationB = { 4, 0 };
+ int[] fieldPermutationB = secondaryFieldPermutationB;
TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp =
- new TreeIndexInsertUpdateDeleteOperatorDescriptor(
- spec, ordersDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, secondaryBloomFilterKeyFields, fieldPermutationB, pipelineOperation,
- dataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
+ new TreeIndexInsertUpdateDeleteOperatorDescriptor(spec, ordersDesc, storageManager, lcManagerProvider,
+ secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
+ secondaryBloomFilterKeyFields, fieldPermutationB, pipelineOperation,
+ secondaryDataflowHelperFactory, null, NoOpOperationCallbackFactory.INSTANCE,
+ pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryInsertOp, NC1_ID);
NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
@@ -326,8 +287,9 @@
protected void destroyPrimaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
- IndexDropOperatorDescriptor primaryDropOp = new IndexDropOperatorDescriptor(spec, storageManager,
- lcManagerProvider, primarySplitProvider, dataflowHelperFactory, pageManagerFactory);
+ IndexDropOperatorDescriptor primaryDropOp =
+ new IndexDropOperatorDescriptor(spec, storageManager, lcManagerProvider, primarySplitProvider,
+ primaryDataflowHelperFactory, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryDropOp, NC1_ID);
spec.addRoot(primaryDropOp);
runTest(spec);
@@ -335,8 +297,9 @@
protected void destroySecondaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
- IndexDropOperatorDescriptor secondaryDropOp = new IndexDropOperatorDescriptor(spec, storageManager,
- lcManagerProvider, secondarySplitProvider, dataflowHelperFactory, pageManagerFactory);
+ IndexDropOperatorDescriptor secondaryDropOp =
+ new IndexDropOperatorDescriptor(spec, storageManager, lcManagerProvider, secondarySplitProvider,
+ secondaryDataflowHelperFactory, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryDropOp, NC1_ID);
spec.addRoot(secondaryDropOp);
runTest(spec);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
index 4b7bac2..605366d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -19,8 +19,6 @@
package org.apache.hyracks.tests.am.btree;
-import java.io.DataOutput;
-
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -36,11 +34,18 @@
import org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import org.junit.Before;
import org.junit.Test;
+import java.io.DataOutput;
+
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryKeyFieldCount;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryRecDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryTypeTraits;
+
public class BTreePrimaryIndexScanOperatorTest extends AbstractBTreeOperatorTest {
@Override
@@ -76,8 +81,8 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
- false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, new LinkedMetadataPageManagerFactory());
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, primaryDataflowHelperFactory, false,
+ false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -92,7 +97,7 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
index 7f1b39b..8874330 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -19,8 +19,6 @@
package org.apache.hyracks.tests.am.btree;
-import java.io.DataOutput;
-
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -41,6 +39,14 @@
import org.junit.Before;
import org.junit.Test;
+import java.io.DataOutput;
+
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryKeyFieldCount;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryRecDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryTypeTraits;
+
public class BTreePrimaryIndexSearchOperatorTest extends AbstractBTreeOperatorTest {
@Override
@@ -81,7 +87,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true, primaryDataflowHelperFactory, false,
false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
@@ -97,7 +103,7 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
index 14e8816..b26071e 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -33,6 +33,10 @@
import org.junit.Before;
import org.junit.Test;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryTypeTraits;
+
public class BTreePrimaryIndexStatsOperatorTest extends AbstractBTreeOperatorTest {
@Override
@@ -49,7 +53,7 @@
TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
- primaryBloomFilterKeyFields, dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE,
+ primaryBloomFilterKeyFields, primaryDataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE,
pageManagerFactory);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -62,7 +66,7 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 9d04dbd..7e04d75 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -41,6 +41,8 @@
import org.junit.Before;
import org.junit.Test;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.*;
+
public class BTreeSecondaryIndexInsertOperatorTest extends AbstractBTreeOperatorTest {
@Override
@@ -86,7 +88,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, dataflowHelperFactory, false, false, null,
+ secondaryHighKeyFields, true, true, primaryDataflowHelperFactory, false, false, null,
NoOpOperationCallbackFactory.INSTANCE, null, null, new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -100,7 +102,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null,
+ primaryDataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null,
new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
@@ -117,7 +119,7 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 49174a9..322e4f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -41,6 +41,8 @@
import org.junit.Before;
import org.junit.Test;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.*;
+
public class BTreeSecondaryIndexSearchOperatorTest extends AbstractBTreeOperatorTest {
@Override
@@ -85,7 +87,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, dataflowHelperFactory, false, false, null,
+ secondaryHighKeyFields, true, true, primaryDataflowHelperFactory, false, false, null,
NoOpOperationCallbackFactory.INSTANCE, null, null, new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -99,7 +101,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null,
+ primaryDataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null,
new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -113,7 +115,7 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
index a355f60..2de100b 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -40,6 +40,8 @@
import org.junit.Before;
import org.junit.Test;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.*;
+
public class BTreeSecondaryIndexUpsertOperatorTest extends AbstractBTreeOperatorTest {
@Override
@@ -85,7 +87,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, lcManagerProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryBloomFilterKeyFields, secondaryLowKeyFields,
- secondaryHighKeyFields, true, true, dataflowHelperFactory, false, false, null,
+ secondaryHighKeyFields, true, true, primaryDataflowHelperFactory, false, false, null,
NoOpOperationCallbackFactory.INSTANCE, null, null, new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
@@ -99,7 +101,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, lcManagerProvider, primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null,
+ primaryDataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null,
new LinkedMetadataPageManagerFactory());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
@@ -116,7 +118,7 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return ((BTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/DataSetConstants.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/DataSetConstants.java
new file mode 100644
index 0000000..d6775cc
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/DataSetConstants.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.tests.am.btree;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+
+public class DataSetConstants {
+
+ public static final RecordDescriptor inputRecordDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
+
+ public static final IValueParserFactory[] inputParserFactories =
+ new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE };
+
+ // field, type and key declarations for primary index
+ public static int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
+ public static final int[] primaryFilterFields = new int[] { 0 };
+ public static final int[] primaryBtreeFields = new int[] { 0, 1, 2, 3, 4, 5 };
+
+ public static final ITypeTraits[] filterTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+ public static final IBinaryComparatorFactory[] filterCmpFactories =
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+ public static final ITypeTraits[] primaryTypeTraits =
+ new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS, UTF8StringPointable.TYPE_TRAITS,
+ UTF8StringPointable.TYPE_TRAITS, UTF8StringPointable.TYPE_TRAITS, UTF8StringPointable.TYPE_TRAITS,
+ UTF8StringPointable.TYPE_TRAITS };
+
+ public static final IBinaryComparatorFactory[] primaryComparatorFactories =
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+ public static final int primaryKeyFieldCount = primaryComparatorFactories.length;
+
+ public static final int[] primaryBloomFilterKeyFields = new int[] { 0 };
+
+ public static final RecordDescriptor primaryRecDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer() });
+
+ public static final RecordDescriptor primaryAndFilterRecDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer() });
+
+ // field, type and key declarations for secondary indexes
+
+ public static final int secondaryKeyFieldCount = 2;
+ public static final int[] secondaryFieldPermutationA = { 3, 0 };
+ public static final int[] secondaryFieldPermutationB = { 4, 0 };
+ public static final int[] secondaryFilterFields = new int[] { 1 };
+ public static final int[] secondaryBtreeFields = new int[] { 0, 1 };
+ public static final int[] secondaryBloomFilterKeyFields = new int[] { 0, 1 };
+
+ public static final ITypeTraits[] secondaryTypeTraits =
+ new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS, UTF8StringPointable.TYPE_TRAITS };
+
+ public static final IBinaryComparatorFactory[] secondaryComparatorFactories =
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+ public static final RecordDescriptor secondaryRecDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer() });
+ public static final RecordDescriptor secondaryWithFilterRecDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer() });
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
index 936bd13..d7a2c18 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeOperatorTestHelper.java
@@ -19,9 +19,6 @@
package org.apache.hyracks.tests.am.lsm.btree;
-import java.util.HashMap;
-import java.util.Map;
-
import org.apache.hyracks.control.nc.io.IOManager;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
@@ -31,9 +28,16 @@
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingOperationTrackerFactory;
import org.apache.hyracks.tests.am.common.LSMTreeOperatorTestHelper;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.filterCmpFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.filterTypeTraits;
+
public class LSMBTreeOperatorTestHelper extends LSMTreeOperatorTestHelper {
private static final Map<String, String> MERGE_POLICY_PROPERTIES;
+
static {
MERGE_POLICY_PROPERTIES = new HashMap<String, String>();
MERGE_POLICY_PROPERTIES.put("num-components", "3");
@@ -43,11 +47,12 @@
super(ioManager);
}
- public IIndexDataflowHelperFactory createDataFlowHelperFactory() {
+ public IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
return new LSMBTreeDataflowHelperFactory(virtualBufferCacheProvider, new ConstantMergePolicyFactory(),
MERGE_POLICY_PROPERTIES, ThreadCountingOperationTrackerFactory.INSTANCE,
SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE,
- DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, null, null, null, null, true);
+ DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, true, filterTypeTraits, filterCmpFactories,
+ btreeFields, filterFields, true);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java
index 839187c..e342a0c 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexScanOperatorTest.java
@@ -33,7 +33,8 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
- return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
+ return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(btreeFields, filterFields);
}
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
index e29465d..fde9aac 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
@@ -19,11 +19,38 @@
package org.apache.hyracks.tests.am.lsm.btree;
+import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.tests.am.btree.BTreePrimaryIndexSearchOperatorTest;
import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import org.apache.hyracks.tests.util.NoopMissingWriterFactory;
+import org.junit.Test;
+
+import java.io.DataOutput;
+
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryAndFilterRecDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryFilterFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryKeyFieldCount;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryTypeTraits;
public class LSMBTreePrimaryIndexSearchOperatorTest extends BTreePrimaryIndexSearchOperatorTest {
@Override
@@ -32,7 +59,114 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
- return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
+ return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(btreeFields, filterFields);
+ }
+
+ @Test
+ public void shouldWriteFilterValueIfAppendFilterIsTrue() 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();
+
+ tb.reset();
+ // low key
+ new UTF8StringSerializerDeserializer().serialize("100", dos);
+ tb.addFieldEndOffset();
+ // high key
+ new UTF8StringSerializerDeserializer().serialize("200", dos);
+ tb.addFieldEndOffset();
+
+ ISerializerDeserializer[] keyRecDescSers =
+ { new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+ ConstantTupleSourceOperatorDescriptor keyProviderOp =
+ new ConstantTupleSourceOperatorDescriptor(spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
+ tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+ int[] lowKeyFields = { 0 };
+ int[] highKeyFields = { 1 };
+
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, primaryAndFilterRecDesc, storageManager, lcManagerProvider,
+ primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
+ primaryDataflowHelperFactory,
+ false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, true,
+ null, null, new LinkedMetadataPageManagerFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ 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.addRoot(printer);
+ runTest(spec);
+ }
+
+ @Test
+ public void shouldWriteNothingIfGivenFilterValueIsOutOfRange() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ // build tuple containing low and high search key
+ // high key and low key
+ ArrayTupleBuilder tb = new ArrayTupleBuilder((primaryKeyFieldCount + primaryFilterFields.length) * 2);
+ DataOutput dos = tb.getDataOutput();
+
+ tb.reset();
+ // low key
+ new UTF8StringSerializerDeserializer().serialize("100", dos);
+ tb.addFieldEndOffset();
+ // high key
+ new UTF8StringSerializerDeserializer().serialize("200", dos);
+ tb.addFieldEndOffset();
+ // min filter
+ new UTF8StringSerializerDeserializer().serialize("9999", dos);
+ tb.addFieldEndOffset();
+ // max filter
+ new UTF8StringSerializerDeserializer().serialize("9999", dos);
+ tb.addFieldEndOffset();
+
+ ISerializerDeserializer[] keyRecDescSers =
+ { new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+ 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[] minFilterFields = { 2 };
+ int[] maxFilterFields = { 3 };
+
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, primaryAndFilterRecDesc, storageManager, lcManagerProvider,
+ primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
+ primaryDataflowHelperFactory,
+ false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, true,
+ minFilterFields, maxFilterFields, new LinkedMetadataPageManagerFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ 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.addRoot(printer);
+ runTest(spec);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java
index f0b4436..c0eca6a 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexInsertOperatorTest.java
@@ -32,7 +32,8 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
- return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
+ return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(btreeFields, filterFields);
}
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
index 22a5700..5306445 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
@@ -19,11 +19,41 @@
package org.apache.hyracks.tests.am.lsm.btree;
+import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.tests.am.btree.BTreeSecondaryIndexSearchOperatorTest;
import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
+import org.apache.hyracks.tests.util.NoopMissingWriterFactory;
+import org.junit.Test;
+
+import java.io.DataOutput;
+
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryRecDesc;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.primaryTypeTraits;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryBloomFilterKeyFields;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryComparatorFactories;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryKeyFieldCount;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryTypeTraits;
+import static org.apache.hyracks.tests.am.btree.DataSetConstants.secondaryWithFilterRecDesc;
public class LSMBTreeSecondaryIndexSearchOperatorTest extends BTreeSecondaryIndexSearchOperatorTest {
@Override
@@ -32,7 +62,71 @@
}
@Override
- protected IIndexDataflowHelperFactory createDataFlowHelperFactory() {
- return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory();
+ protected IIndexDataflowHelperFactory createDataFlowHelperFactory(int[] btreeFields, int[] filterFields) {
+ return ((LSMBTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(btreeFields, filterFields);
}
+
+ @Test
+ public void shouldCarryFilterValueToPrimaryIndexSearch() 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();
+
+ tb.reset();
+ // low key
+ new UTF8StringSerializerDeserializer().serialize("1998-07-21", dos);
+ tb.addFieldEndOffset();
+ // high key
+ new UTF8StringSerializerDeserializer().serialize("2000-10-18", dos);
+ tb.addFieldEndOffset();
+
+ ISerializerDeserializer[] keyRecDescSers =
+ { new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() };
+ RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
+
+ ConstantTupleSourceOperatorDescriptor keyProviderOp =
+ new ConstantTupleSourceOperatorDescriptor(spec, keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(),
+ tb.getSize());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, NC1_ID);
+
+ int[] secondaryLowKeyFields = { 0 };
+ int[] secondaryHighKeyFields = { 1 };
+
+ // search secondary index
+ BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, secondaryWithFilterRecDesc, storageManager, lcManagerProvider,
+ secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
+ secondaryBloomFilterKeyFields, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+ primaryDataflowHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
+ NoOpOperationCallbackFactory.INSTANCE, true, null, null,
+ new LinkedMetadataPageManagerFactory());
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
+
+ int[] primaryLowKeyFields = { 1 }; // second field from the tuples
+ int[] primaryHighKeyFields = { 1 }; // second field from the tuples
+ int[] minFilterFields = { 2 };
+ int[] maxFilterFields = { 3 };
+
+ // search primary index
+ BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+ new BTreeSearchOperatorDescriptor(spec, primaryRecDesc, storageManager, lcManagerProvider,
+ primarySplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ primaryBloomFilterKeyFields, primaryLowKeyFields, primaryHighKeyFields, true, true,
+ primaryDataflowHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE,
+ minFilterFields, maxFilterFields, new LinkedMetadataPageManagerFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ 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.addRoot(printer);
+ runTest(spec);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
index f4c80c1..4eb3f4b 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeOperatorTestHelper.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.control.nc.io.IOManager;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -50,11 +51,12 @@
public IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) {
return new LSMRTreeDataflowHelperFactory(valueProviderFactories, rtreePolicyType, btreeComparatorFactories,
virtualBufferCacheProvider, new ConstantMergePolicyFactory(), MERGE_POLICY_PROPERTIES,
ThreadCountingOperationTrackerFactory.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
- NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, null,
- btreeFields, null, null, null, true, false);
+ NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, rtreeFields,
+ btreeFields, filterTypeTraits, filterCmpFactories, filterFields, true, false);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java
index b3b178d..0f03ec8 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexInsertOperatorTest.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -44,8 +45,10 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
- return ((LSMRTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
- rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory, btreeFields);
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) {
+ return ((LSMRTreeOperatorTestHelper) testHelper)
+ .createDataFlowHelperFactory(secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories,
+ linearizerCmpFactory, btreeFields, rtreeFields, filterTypeTraits, filterCmpFactories, filterFields);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java
index 18b8c77..e9d4605 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeSecondaryIndexSearchOperatorTest.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -43,8 +44,10 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
- return ((LSMRTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
- rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory, btreeFields);
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) {
+ return ((LSMRTreeOperatorTestHelper) testHelper)
+ .createDataFlowHelperFactory(secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories,
+ linearizerCmpFactory, btreeFields, rtreeFields, filterTypeTraits, filterCmpFactories, filterFields);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
index a2a6971..752cfd9 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesOperatorTestHelper.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.control.nc.io.IOManager;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -38,6 +39,7 @@
public class LSMRTreeWithAntiMatterTuplesOperatorTestHelper extends LSMTreeOperatorTestHelper {
private static final Map<String, String> MERGE_POLICY_PROPERTIES;
+
static {
MERGE_POLICY_PROPERTIES = new HashMap<String, String>();
MERGE_POLICY_PROPERTIES.put("num-components", "3");
@@ -49,11 +51,13 @@
public IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
- IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory) {
+ IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
+ int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) {
return new LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(valueProviderFactories, rtreePolicyType,
btreeComparatorFactories, virtualBufferCacheProvider, new ConstantMergePolicyFactory(),
MERGE_POLICY_PROPERTIES, ThreadCountingOperationTrackerFactory.INSTANCE,
- SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory, null,
- null, null, null, true, false);
+ SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, linearizerCmpFactory,
+ rtreeFields, filterTypeTraits, filterCmpFactories, filterFields, true, false);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java
index dea5e6f..668c4ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest.java
@@ -21,6 +21,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -29,7 +30,8 @@
import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
import org.apache.hyracks.tests.am.rtree.RTreeSecondaryIndexInsertOperatorTest;
-public class LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest extends RTreeSecondaryIndexInsertOperatorTest {
+public class LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest
+ extends RTreeSecondaryIndexInsertOperatorTest {
public LSMRTreeWithAntiMatterTuplesSecondaryIndexInsertOperatorTest() {
this.rTreeType = RTreeType.LSMRTREE_WITH_ANTIMATTER;
}
@@ -43,8 +45,10 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
- return ((LSMRTreeWithAntiMatterTuplesOperatorTestHelper) testHelper).createDataFlowHelperFactory(
- secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory);
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
+ IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) {
+ return ((LSMRTreeWithAntiMatterTuplesOperatorTestHelper) testHelper)
+ .createDataFlowHelperFactory(secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories,
+ linearizerCmpFactory, rtreeFields, filterTypeTraits, filterCmpFactories, filterFields);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
index b725820..4d2e0a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
@@ -19,20 +19,38 @@
package org.apache.hyracks.tests.am.lsm.rtree;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.*;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.tests.am.common.ITreeIndexOperatorTestHelper;
import org.apache.hyracks.tests.am.rtree.RTreeSecondaryIndexSearchOperatorTest;
+import org.apache.hyracks.tests.util.NoopMissingWriterFactory;
+import org.junit.Test;
-public class LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest extends RTreeSecondaryIndexSearchOperatorTest {
+import java.io.DataOutput;
+
+public class LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest
+ extends RTreeSecondaryIndexSearchOperatorTest {
public LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest() {
- this.rTreeType = RTreeType.LSMRTREE_WITH_ANTIMATTER;
- }
+ this.rTreeType = RTreeType.LSMRTREE_WITH_ANTIMATTER;
+ }
@Override
protected ITreeIndexOperatorTestHelper createTestHelper() throws HyracksDataException {
@@ -43,8 +61,50 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
- return ((LSMRTreeWithAntiMatterTuplesOperatorTestHelper) testHelper).createDataFlowHelperFactory(
- secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories, linearizerCmpFactory);
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits,
+ IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields) {
+ return ((LSMRTreeWithAntiMatterTuplesOperatorTestHelper) testHelper)
+ .createDataFlowHelperFactory(secondaryValueProviderFactories, rtreePolicyType, btreeComparatorFactories,
+ linearizerCmpFactory, rtreeFields, filterTypeTraits, filterCmpFactories, filterFields);
+ }
+
+ @Test
+ public void shouldWriteFilterValueIfAppendFilterIsTrue() 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 secondarySearchOp =
+ new RTreeSearchOperatorDescriptor(spec, secondaryWithFilterRecDesc, storageManager, lcManagerProvider,
+ secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories, keyFields,
+ rtreeDataflowHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
+ NoOpOperationCallbackFactory.INSTANCE, true, null, null, pageManagerFactory);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
+
+ IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+ spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, printer, 0);
+ spec.addRoot(printer);
+ runTest(spec);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index ac69a5a..0d4bf92 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -19,9 +19,6 @@
package org.apache.hyracks.tests.am.rtree;
-import java.io.DataOutput;
-import java.io.File;
-
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -59,6 +56,7 @@
import org.apache.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.freepage.LinkedMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -75,6 +73,9 @@
import org.junit.After;
import org.junit.Before;
+import java.io.DataOutput;
+import java.io.File;
+
public abstract class AbstractRTreeOperatorTest extends AbstractIntegrationTest {
static {
TestStorageManagerComponentHolder.init(8192, 20, 20);
@@ -90,7 +91,7 @@
protected final IStorageManager storageManager = new TestStorageManager();
protected final IIndexLifecycleManagerProvider lcManagerProvider = new TestIndexLifecycleManagerProvider();
- protected final IPageManagerFactory pageManagerFactory = new LinkedMetadataPageManagerFactory();
+ protected final IPageManagerFactory pageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
protected IIndexDataflowHelperFactory rtreeDataflowHelperFactory;
protected IIndexDataflowHelperFactory btreeDataflowHelperFactory = new BTreeDataflowHelperFactory(true);
@@ -122,6 +123,12 @@
DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
new UTF8StringSerializerDeserializer() });
+ protected final RecordDescriptor secondaryWithFilterRecDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ DoubleSerializerDeserializer.INSTANCE, DoubleSerializerDeserializer.INSTANCE,
+ new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+ new UTF8StringSerializerDeserializer() });
+
// This is only used for the LSMRTree. We need a comparator Factories for
// the BTree component of the LSMRTree.
protected int btreeKeyFieldCount = 5;
@@ -141,11 +148,11 @@
testHelper = createTestHelper();
primaryFileName = testHelper.getPrimaryIndexName();
- primarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID,
- primaryFileName) });
+ primarySplitProvider =
+ new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID, primaryFileName) });
secondaryFileName = testHelper.getSecondaryIndexName();
- secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID,
- secondaryFileName) });
+ secondarySplitProvider =
+ new ConstantFileSplitProvider(new FileSplit[] { new ManagedFileSplit(NC1_ID, secondaryFileName) });
// field, type and key declarations for primary index
primaryTypeTraits[0] = UTF8StringPointable.TYPE_TRAITS;
@@ -172,14 +179,24 @@
secondaryComparatorFactories[3] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
// This only used for LSMRTree
+ int[] rtreeFields = null;
+ int[] filterFields = null;
+ ITypeTraits[] filterTypes = null;
+ IBinaryComparatorFactory[] filterCmpFactories = null;
+ if (rTreeType == RTreeType.LSMRTREE || rTreeType == RTreeType.LSMRTREE_WITH_ANTIMATTER) {
+ rtreeFields = new int[] { 0, 1, 2, 3, 4 };
+ filterFields = new int[] { 4 };
+ filterTypes = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
+ filterCmpFactories =
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+ }
+
int[] btreeFields = null;
if (rTreeType == RTreeType.LSMRTREE) {
btreeKeyFieldCount = 1;
btreeComparatorFactories = new IBinaryComparatorFactory[btreeKeyFieldCount];
btreeComparatorFactories[0] = PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY);
- btreeFields = new int[1];
- btreeFields[0] = 4;
-
+ btreeFields = new int[] { 4 };
} else {
btreeComparatorFactories[0] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
btreeComparatorFactories[1] = PointableBinaryComparatorFactory.of(DoublePointable.FACTORY);
@@ -191,17 +208,19 @@
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories = RTreeUtils
.createPrimitiveValueProviderFactories(secondaryComparatorFactories.length, DoublePointable.FACTORY);
- rtreeDataflowHelperFactory = createDataFlowHelperFactory(secondaryValueProviderFactories,
- RTreePolicyType.RSTARTREE, btreeComparatorFactories,
- LSMRTreeUtils.proposeBestLinearizer(secondaryTypeTraits, secondaryComparatorFactories.length),
- btreeFields);
+ rtreeDataflowHelperFactory =
+ createDataFlowHelperFactory(secondaryValueProviderFactories, RTreePolicyType.RSTARTREE,
+ btreeComparatorFactories,
+ LSMRTreeUtils.proposeBestLinearizer(secondaryTypeTraits, secondaryComparatorFactories.length),
+ btreeFields, rtreeFields, filterTypes, filterCmpFactories, filterFields);
}
protected abstract IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) throws HyracksDataException;
+ int[] btreeFields, int[] rtree, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) throws HyracksDataException;
protected void createPrimaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
index 0296a35..3fe64c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -23,10 +23,7 @@
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.*;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileSplit;
import org.apache.hyracks.api.job.JobSpecification;
@@ -116,7 +113,8 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) throws HyracksDataException {
+ int[] btreeFields, int [] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) throws HyracksDataException {
return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
rtreePolicyType, null, true);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
index 3b42f55..0cf9c52 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -23,10 +23,7 @@
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.*;
import org.apache.hyracks.api.io.FileSplit;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -101,7 +98,8 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) {
return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
rtreePolicyType, null, true);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 092aa9c..8818282 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -23,10 +23,7 @@
import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.*;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileSplit;
import org.apache.hyracks.api.job.JobSpecification;
@@ -114,7 +111,8 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) throws HyracksDataException {
+ int[] btreeFields, int [] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) throws HyracksDataException {
return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
rtreePolicyType, null, true);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java
index a9e2997..59a57f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexStatsOperatorTest.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.io.FileSplit;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -71,7 +72,8 @@
protected IIndexDataflowHelperFactory createDataFlowHelperFactory(
IPrimitiveValueProviderFactory[] secondaryValueProviderFactories, RTreePolicyType rtreePolicyType,
IBinaryComparatorFactory[] btreeComparatorFactories, ILinearizeComparatorFactory linearizerCmpFactory,
- int[] btreeFields) {
+ int[] btreeFields, int[] rtreeFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields) {
return ((RTreeOperatorTestHelper) testHelper).createDataFlowHelperFactory(secondaryValueProviderFactories,
rtreePolicyType, null, true);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index acb008d..7a98fb8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -48,6 +48,7 @@
protected final boolean highKeyInclusive;
private final int[] minFilterFieldIndexes;
private final int[] maxFilterFieldIndexes;
+ private boolean appendFilter;
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
@@ -56,7 +57,8 @@
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchOpCallbackProvider,
- int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IPageManagerFactory pageManagerFactory) {
+ boolean appendFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+ IPageManagerFactory pageManagerFactory) {
super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
comparatorFactories, bloomFilterKeyFields, dataflowHelperFactory, null, retainInput, retainMissing,
missingWriterFactory, NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackProvider,
@@ -67,12 +69,28 @@
this.highKeyInclusive = highKeyInclusive;
this.minFilterFieldIndexes = minFilterFieldIndexes;
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
+ this.appendFilter = appendFilter;
+ }
+
+ public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+ IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, int[] bloomFilterKeyFields, int[] lowKeyFields,
+ int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainMissing,
+ IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchOpCallbackProvider,
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IPageManagerFactory pageManagerFactory) {
+ this(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields, lowKeyInclusive,
+ highKeyInclusive, dataflowHelperFactory, retainInput, retainMissing, missingWriterFactory,
+ searchOpCallbackProvider, false, minFilterFieldIndexes, maxFilterFieldIndexes, pageManagerFactory);
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
return new BTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, lowKeyFields,
- highKeyFields, lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes);
+ highKeyFields, lowKeyInclusive, highKeyInclusive, appendFilter,
+ minFilterFieldIndexes, maxFilterFieldIndexes);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index c7af3fe..64be47e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -41,9 +41,9 @@
public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, int[] lowKeyFields, int[] highKeyFields,
- boolean lowKeyInclusive, boolean highKeyInclusive, int[] minFilterFieldIndexes,
+ boolean lowKeyInclusive, boolean highKeyInclusive, boolean appendFilter, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes) throws HyracksDataException {
- super(opDesc, ctx, partition, recordDescProvider, minFilterFieldIndexes, maxFilterFieldIndexes);
+ super(opDesc, ctx, partition, recordDescProvider, appendFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
if (lowKeyFields != null && lowKeyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index 7b81585..023bc81 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -39,7 +39,7 @@
int partition, IRecordDescriptorProvider recordDescProvider, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, ITupleUpdater tupleUpdater) throws HyracksDataException {
super(opDesc, ctx, partition, recordDescProvider, lowKeyFields, highKeyFields, lowKeyInclusive,
- highKeyInclusive, null, null);
+ highKeyInclusive, false, null, null);
this.tupleUpdater = tupleUpdater;
}
@@ -68,8 +68,7 @@
dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
tb.addFieldEndOffset();
}
- FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize());
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
index a02b492..4e1da71 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java
@@ -234,6 +234,16 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
+ @Override
public ICachedPage getPage() {
return page;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
index 5452604..4e82858 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java
@@ -103,6 +103,16 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
+ @Override
public ICachedPage getPage() {
return page;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 75b2fdd..b6be7a8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -285,7 +285,7 @@
for (IRecordDescriptorProvider recordDescProvider : recordDescProviders) {
for (IHyracksTaskContext ctx : ctxs) {
BTreeSearchOperatorNodePushable writer = new BTreeSearchOperatorNodePushable(opDesc, ctx, partition,
- recordDescProvider, keys, keys, lowKeyInclusive, highKeyInclusive, keys, keys);
+ recordDescProvider, keys, keys, lowKeyInclusive, highKeyInclusive, false, keys, keys);
writers.add(writer);
}
}
@@ -399,6 +399,7 @@
// normal
indexHelpers[j] = Mockito.mock(IIndexDataflowHelper.class);
Mockito.when(indexHelpers[j].getIndexInstance()).thenReturn(indexes[i]);
+ Mockito.when(indexHelpers[j].getNumFilterFields()).thenReturn(0);
// throws exception when opened
j++;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexCursor.java
index 2b5f3f6..5264f9e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexCursor.java
@@ -67,4 +67,15 @@
* @return the tuple pointed to by the cursor
*/
ITupleReference getTuple();
+
+ /**
+ * @return the min tuple of the current index's filter
+ */
+ ITupleReference getFilterMinTuple();
+
+ /**
+ *
+ * @return the max tuple of the current index's filter
+ */
+ ITupleReference getFilterMaxTuple();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
index 261543a..1bd6673 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
@@ -40,4 +40,6 @@
public IIndex getIndexInstance();
public LocalResource getResource() throws HyracksDataException;
+
+ public int getNumFilterFields();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
deleted file mode 100644
index 0a41e47..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptor.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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 at
- *
- * 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 org.apache.hyracks.storage.am.common.dataflow;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITypeTraits;
-import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
-
-public interface ITreeIndexOperatorDescriptor extends IIndexOperatorDescriptor {
- public IBinaryComparatorFactory[] getTreeIndexComparatorFactories();
-
- public ITypeTraits[] getTreeIndexTypeTraits();
-
- public ITupleFilterFactory getTupleFilterFactory();
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index c089854..3ce9339 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -21,6 +21,8 @@
import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -45,6 +47,7 @@
import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+ static final Logger LOGGER = Logger.getLogger(IndexSearchOperatorNodePushable.class.getName());
protected final IIndexOperatorDescriptor opDesc;
protected final IHyracksTaskContext ctx;
protected final IIndexDataflowHelper indexHelper;
@@ -71,16 +74,20 @@
protected final int[] maxFilterFieldIndexes;
protected PermutingFrameTupleReference minFilterKey;
protected PermutingFrameTupleReference maxFilterKey;
+ protected final boolean appendIndexFilter;
+ protected ArrayTupleBuilder nonFilterTupleBuild;
+ protected final int numIndexFilterFields;
public IndexSearchOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
- IRecordDescriptorProvider recordDescProvider, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes)
- throws HyracksDataException {
+ IRecordDescriptorProvider recordDescProvider, boolean appendIndexFilter, int[] minFilterFieldIndexes,
+ int[] maxFilterFieldIndexes) throws HyracksDataException {
this.opDesc = opDesc;
this.ctx = ctx;
this.indexHelper = opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(opDesc, ctx, partition);
this.retainInput = opDesc.getRetainInput();
this.retainMissing = opDesc.getRetainMissing();
- if (this.retainMissing) {
+ this.appendIndexFilter = appendIndexFilter;
+ if (this.retainMissing || this.appendIndexFilter) {
this.nonMatchWriter = opDesc.getMissingWriterFactory().createMissingWriter();
}
this.inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
@@ -94,6 +101,7 @@
maxFilterKey = new PermutingFrameTupleReference();
maxFilterKey.setFieldPermutation(maxFilterFieldIndexes);
}
+ this.numIndexFilterFields = indexHelper.getNumFilterFields();
}
protected abstract ISearchPredicate createSearchPredicate();
@@ -115,19 +123,16 @@
if (retainMissing) {
int fieldCount = getFieldCount();
nonMatchTupleBuild = new ArrayTupleBuilder(fieldCount);
- DataOutput out = nonMatchTupleBuild.getDataOutput();
- for (int i = 0; i < fieldCount; i++) {
- try {
- nonMatchWriter.writeMissing(out);
- } catch (IOException e) {
- e.printStackTrace();
- }
- nonMatchTupleBuild.addFieldEndOffset();
- }
+ buildMissingTuple(fieldCount, nonMatchTupleBuild, nonMatchWriter);
} else {
nonMatchTupleBuild = null;
}
+ if (appendIndexFilter) {
+ nonFilterTupleBuild = new ArrayTupleBuilder(numIndexFilterFields);
+ buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
+ }
+
try {
searchPred = createSearchPredicate();
tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
@@ -159,9 +164,10 @@
}
}
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();
+ writeTupleToOutput(tuple);
+ if (appendIndexFilter) {
+ writeFilterTupleToOutput(cursor.getFilterMinTuple());
+ writeFilterTupleToOutput(cursor.getFilterMaxTuple());
}
FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
}
@@ -245,4 +251,36 @@
public void fail() throws HyracksDataException {
writer.fail();
}
+
+ private void writeTupleToOutput(ITupleReference tuple) throws IOException {
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ }
+
+ private void writeFilterTupleToOutput(ITupleReference tuple) throws IOException {
+ if (tuple != null) {
+ writeTupleToOutput(tuple);
+ } else {
+ int[] offsets = nonFilterTupleBuild.getFieldEndOffsets();
+ for (int i = 0; i < offsets.length; i++) {
+ int start = i > 0 ? offsets[i - 1] : 0;
+ tb.addField(nonFilterTupleBuild.getByteArray(), start, offsets[i]);
+ }
+ }
+ }
+
+ private static void buildMissingTuple(int numFields, ArrayTupleBuilder nullTuple, IMissingWriter nonMatchWriter) {
+ DataOutput out = nullTuple.getDataOutput();
+ for (int i = 0; i < numFields; i++) {
+ try {
+ nonMatchWriter.writeMissing(out);
+ } catch (Exception e) {
+ LOGGER.log(Level.WARNING, e.getMessage(), e);
+ }
+ nullTuple.addFieldEndOffset();
+ }
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
index 07cab2e..b7a6df9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDataflowHelper.java
@@ -26,6 +26,10 @@
import org.apache.hyracks.storage.am.common.impls.TreeIndexDiskOrderScanCursor;
public abstract class TreeIndexDataflowHelper extends IndexDataflowHelper {
+ @Override
+ public int getNumFilterFields() {
+ return 0;
+ }
public TreeIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
boolean durable) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
index 679ea2f..6bd3e60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.storage.am.common.impls;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.ICursorInitialState;
import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
@@ -59,6 +60,16 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
+ @Override
public ICachedPage getPage() {
return page;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
index 99c3e3f..95292dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java
@@ -33,11 +33,8 @@
import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.*;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
import org.apache.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
@@ -54,6 +51,7 @@
private ILSMHarness lsmHarness;
private boolean nextHasBeenCalled;
private boolean foundTuple;
+ private int foundIn = -1;
private ITupleReference frameTuple;
private List<ILSMComponent> operationalComponents;
@@ -86,6 +84,7 @@
} else {
frameTuple = rangeCursors[i].getTuple();
foundTuple = true;
+ foundIn = i;
return true;
}
}
@@ -107,6 +106,7 @@
frameTuple = rangeCursors[i].getTuple();
foundTuple = true;
searchCallback.complete(predicate.getLowKey());
+ foundIn = i;
return true;
}
} else {
@@ -118,6 +118,7 @@
searchCallback.reconcile(frameTuple);
searchCallback.complete(frameTuple);
foundTuple = true;
+ foundIn = i;
return true;
}
} else {
@@ -228,6 +229,25 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
+ private ILSMComponentFilter getFilter() {
+ if (foundTuple) {
+ return operationalComponents.get(foundIn).getLSMComponentFilter();
+ }
+ return null;
+ }
+
+ @Override
public ICachedPage getPage() {
// do nothing
return null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index a368ee9..937b659 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -86,6 +86,16 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ return currentCursor.getFilterMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return currentCursor.getFilterMaxTuple();
+ }
+
+ @Override
public ICachedPage getPage() {
return currentCursor.getPage();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
index 768ef27..540f5a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySearchCursor.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.storage.am.common.api.ICursorInitialState;
import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
public class LSMBTreeWithBuddySearchCursor extends LSMBTreeWithBuddyAbstractCursor {
@@ -112,6 +113,25 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
+ private ILSMComponentFilter getFilter() {
+ if (currentCursor < 0) {
+ return null;
+ }
+ return operationalComponents.get(currentCursor).getLSMComponentFilter();
+ }
+
+ @Override
public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
super.open(initialState, searchPred);
searchNextCursor();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
index 5e356c4..4d45349 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddySortedCursor.java
@@ -19,9 +19,11 @@
package org.apache.hyracks.storage.am.lsm.btree.impls;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.ICursorInitialState;
import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
public class LSMBTreeWithBuddySortedCursor extends LSMBTreeWithBuddyAbstractCursor {
@@ -69,6 +71,25 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
+ private ILSMComponentFilter getFilter() {
+ if (foundIn < 0) {
+ return null;
+ }
+ return operationalComponents.get(foundIn).getLSMComponentFilter();
+ }
+
+ @Override
public boolean hasNext() throws HyracksDataException {
while (!foundNext) {
frameTuple = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
index f219c12..9b24e24 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.api.IIndex;
import org.apache.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
@@ -76,4 +77,9 @@
this.filterCmpFactories = filterCmpFactories;
this.filterFields = filterFields;
}
+
+ @Override
+ public int getNumFilterFields() {
+ return filterFields == null ? 0 : filterFields.length;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index b8d21e4..58f0c7a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -28,10 +28,7 @@
import org.apache.hyracks.storage.am.common.api.IIndexCursor;
import org.apache.hyracks.storage.am.common.api.ITreeIndexCursor;
import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMTreeTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.*;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
@@ -169,6 +166,18 @@
return outputElement.getTuple();
}
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = operationalComponents.get(outputElement.cursorIndex).getLSMComponentFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = operationalComponents.get(outputElement.cursorIndex).getLSMComponentFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
protected boolean pushIntoQueueFromCursorAndReplaceThisElement(PriorityQueueElement e) throws HyracksDataException {
int cursorIndex = e.getCursorIndex();
if (rangeCursors[cursorIndex].hasNext()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index f628e76..f0ebedd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -47,6 +47,7 @@
private final IInvertedIndexSearchModifierFactory searchModifierFactory;
private final int[] minFilterFieldIndexes;
private final int[] maxFilterFieldIndexes;
+ private final boolean appendFilter;
private final boolean isFullTextSearchQuery;
public LSMInvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, int queryField,
@@ -57,7 +58,7 @@
IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc, boolean retainInput,
boolean retainNull, IMissingWriterFactory nullWriterFactory,
- ISearchOperationCallbackFactory searchOpCallbackProvider, int[] minFilterFieldIndexes,
+ ISearchOperationCallbackFactory searchOpCallbackProvider, boolean appendFilter, int[] minFilterFieldIndexes,
int[] maxFilterFieldIndexes, IPageManagerFactory pageManagerFactory, boolean isFullTextSearchQuery) {
super(spec, 1, 1, recDesc, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, queryTokenizerFactory,
@@ -69,6 +70,24 @@
this.minFilterFieldIndexes = minFilterFieldIndexes;
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.isFullTextSearchQuery = isFullTextSearchQuery;
+ this.appendFilter = appendFilter;
+ }
+
+ public LSMInvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, int queryField,
+ IStorageManager storageManager, IFileSplitProvider fileSplitProvider,
+ IIndexLifecycleManagerProvider lifecycleManagerProvider, ITypeTraits[] tokenTypeTraits,
+ IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
+ IBinaryComparatorFactory[] invListComparatorFactories,
+ IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
+ IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc, boolean retainInput,
+ boolean retainNull, IMissingWriterFactory nullWriterFactory,
+ ISearchOperationCallbackFactory searchOpCallbackProvider, int[] minFilterFieldIndexes,
+ int[] maxFilterFieldIndexes, IPageManagerFactory pageManagerFactory, boolean isFullTextSearchQuery) {
+ this(spec, queryField, storageManager, fileSplitProvider, lifecycleManagerProvider, tokenTypeTraits,
+ tokenComparatorFactories, invListsTypeTraits, invListComparatorFactories, btreeDataflowHelperFactory,
+ queryTokenizerFactory, searchModifierFactory, recDesc, retainInput, retainNull, nullWriterFactory,
+ searchOpCallbackProvider, false, minFilterFieldIndexes, maxFilterFieldIndexes, pageManagerFactory,
+ isFullTextSearchQuery);
}
@Override
@@ -76,6 +95,6 @@
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
IInvertedIndexSearchModifier searchModifier = searchModifierFactory.createSearchModifier();
return new LSMInvertedIndexSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, queryField,
- searchModifier, minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery);
+ searchModifier, appendFilter, minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index 4634c7f..a5dee83 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -40,10 +40,9 @@
public LSMInvertedIndexSearchOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, int queryFieldIndex,
- IInvertedIndexSearchModifier searchModifier, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
- boolean isFullTextSearchQuery)
- throws HyracksDataException {
- super(opDesc, ctx, partition, recordDescProvider, minFilterFieldIndexes, maxFilterFieldIndexes);
+ IInvertedIndexSearchModifier searchModifier, boolean appendFilter, int[] minFilterFieldIndexes,
+ int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery) throws HyracksDataException {
+ super(opDesc, ctx, partition, recordDescProvider, appendFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
this.searchModifier = searchModifier;
this.queryFieldIndex = queryFieldIndex;
this.isFullTextSearchQuery = isFullTextSearchQuery;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index 9352fbb..70828c4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.storage.am.common.ophelpers.MultiComparator;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.impls.BloomFilterAwareBTreePointSearchCursor;
@@ -190,4 +191,23 @@
public ITupleReference getTuple() {
return currentCursor.getTuple();
}
+
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = getComponentFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = getComponentFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
+ private ILSMComponentFilter getComponentFilter() {
+ if (accessorIndex < 0) {
+ return null;
+ }
+ return operationalComponents.get(accessorIndex).getLSMComponentFilter();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
index 4a74833..4340dec 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -130,4 +130,14 @@
public ITupleReference getTuple() {
return concatTuple;
}
+
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
index 12dd319..1b6d492 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchCursor.java
@@ -94,6 +94,16 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
+ @Override
public void reset() {
currentBufferIndex = 0;
tupleIndex = 0;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index 08458d4..b04c2fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -24,6 +24,8 @@
import org.apache.hyracks.storage.am.common.api.ICursorInitialState;
import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
public class LSMRTreeSearchCursor extends LSMRTreeAbstractCursor {
@@ -63,6 +65,22 @@
}
}
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
+ private ILSMComponentFilter getFilter() {
+ return foundNext ? operationalComponents.get(currentCursor).getLSMComponentFilter() : null;
+ }
+
private void searchNextCursor() throws HyracksDataException {
if (currentCursor < numberOfTrees) {
rtreeCursors[currentCursor].reset();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
index c536712..8252d14 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSortedCursor.java
@@ -22,9 +22,11 @@
import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.ICursorInitialState;
import org.apache.hyracks.storage.am.common.api.ISearchPredicate;
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
public class LSMRTreeSortedCursor extends LSMRTreeAbstractCursor {
@@ -70,6 +72,21 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMinTuple();
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ ILSMComponentFilter filter = getFilter();
+ return filter == null ? null : filter.getMaxTuple();
+ }
+
+ private ILSMComponentFilter getFilter() {
+ return foundIn < 0 ? null : operationalComponents.get(foundIn).getLSMComponentFilter();
+ }
+ @Override
public boolean hasNext() throws HyracksDataException {
while (!foundNext) {
frameTuple = null;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
index d606f77..f2be5a0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java
@@ -145,6 +145,16 @@
}
@Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
+ @Override
public ICachedPage getPage() {
return null;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
index 70b1141..4f894e5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java
@@ -96,6 +96,16 @@
return frameTuple1;
}
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
public void insertTupleEntry(int pageId, int tupleOffset) {
if (numTuples * 2 == tPointers.length) {
int[] newData = new int[tPointers.length + ARRAY_GROWTH];
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index fad00ca..85639ad 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -45,6 +45,25 @@
protected int[] keyFields; // fields in input tuple to be used as keys
protected final int[] minFilterFieldIndexes;
protected final int[] maxFilterFieldIndexes;
+ protected final boolean appendFilter;
+
+ public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
+ IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
+ IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
+ IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
+ IMissingWriterFactory nullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackFactory,
+ boolean appendFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+ IPageManagerFactory pageManagerFactory) {
+ super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, null, dataflowHelperFactory, null, retainInput, retainNull, nullWriterFactory,
+ NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackFactory,
+ NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
+ this.keyFields = keyFields;
+ this.minFilterFieldIndexes = minFilterFieldIndexes;
+ this.maxFilterFieldIndexes = maxFilterFieldIndexes;
+ this.appendFilter = appendFilter;
+ }
public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
IStorageManager storageManager, IIndexLifecycleManagerProvider lifecycleManagerProvider,
@@ -53,19 +72,15 @@
IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, boolean retainNull,
IMissingWriterFactory nullWriterFactory, ISearchOperationCallbackFactory searchOpCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IPageManagerFactory pageManagerFactory) {
- super(spec, 1, 1, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
- comparatorFactories, null, dataflowHelperFactory, null, retainInput, retainNull, nullWriterFactory,
- NoOpLocalResourceFactoryProvider.INSTANCE, searchOpCallbackFactory,
- NoOpOperationCallbackFactory.INSTANCE, pageManagerFactory);
- this.keyFields = keyFields;
- this.minFilterFieldIndexes = minFilterFieldIndexes;
- this.maxFilterFieldIndexes = maxFilterFieldIndexes;
+ this(spec, recDesc, storageManager, lifecycleManagerProvider, fileSplitProvider, typeTraits,
+ comparatorFactories, keyFields, dataflowHelperFactory, retainInput, retainNull, nullWriterFactory,
+ searchOpCallbackFactory, false, minFilterFieldIndexes, maxFilterFieldIndexes, pageManagerFactory);
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
- return new RTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields,
+ return new RTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields, appendFilter,
minFilterFieldIndexes, maxFilterFieldIndexes);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 1dc31b1..d0d77d9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -36,9 +36,9 @@
protected MultiComparator cmp;
public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
- int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes) throws HyracksDataException {
- super(opDesc, ctx, partition, recordDescProvider, minFilterFieldIndexes, maxFilterFieldIndexes);
+ int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields, boolean appendFilter,
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes) throws HyracksDataException {
+ super(opDesc, ctx, partition, recordDescProvider, appendFilter, minFilterFieldIndexes, maxFilterFieldIndexes);
if (keyFields != null && keyFields.length > 0) {
searchKey = new PermutingFrameTupleReference();
searchKey.setFieldPermutation(keyFields);
@@ -67,6 +67,6 @@
@Override
protected int getFieldCount() {
- return ((ITreeIndex)index).getFieldCount();
+ return ((ITreeIndex) index).getFieldCount();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
index 5ee8700..218d34d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java
@@ -79,6 +79,16 @@
return frameTuple;
}
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
public int getTupleOffset() {
return leafFrame.getTupleOffset(currentTupleIndex);
}